You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by al...@apache.org on 2018/10/16 04:18:31 UTC

[01/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Repository: asterixdb
Updated Branches:
  refs/heads/master adfb63361 -> 80225e2c2


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/ForwardOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/ForwardOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/ForwardOperatorDescriptor.java
new file mode 100644
index 0000000..24c5cae
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/misc/ForwardOperatorDescriptor.java
@@ -0,0 +1,246 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.dataflow.std.misc;
+
+import java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.nio.ByteBuffer;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.ActivityId;
+import org.apache.hyracks.api.dataflow.IActivityGraphBuilder;
+import org.apache.hyracks.api.dataflow.IOperatorNodePushable;
+import org.apache.hyracks.api.dataflow.TaskId;
+import org.apache.hyracks.api.dataflow.value.IRecordDescriptorProvider;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IOperatorDescriptorRegistry;
+import org.apache.hyracks.api.job.JobId;
+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.marshalling.ByteArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
+import org.apache.hyracks.dataflow.std.base.AbstractActivityNode;
+import org.apache.hyracks.dataflow.std.base.AbstractOperatorDescriptor;
+import org.apache.hyracks.dataflow.std.base.AbstractStateObject;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputSinkOperatorNodePushable;
+import org.apache.hyracks.dataflow.std.base.AbstractUnaryInputUnaryOutputOperatorNodePushable;
+
+public class ForwardOperatorDescriptor extends AbstractOperatorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private static final int FORWARD_DATA_ACTIVITY_ID = 0;
+    private static final int RANGEMAP_READER_ACTIVITY_ID = 1;
+    private final String rangeMapKeyInContext;
+
+    /**
+     * @param spec used to create the operator id.
+     * @param rangeMapKeyInContext the unique key to store the range map in the shared map & transfer it to partitioner.
+     * @param outputRecordDescriptor the output schema of this operator.
+     */
+    public ForwardOperatorDescriptor(IOperatorDescriptorRegistry spec, String rangeMapKeyInContext,
+            RecordDescriptor outputRecordDescriptor) {
+        super(spec, 2, 1);
+        this.rangeMapKeyInContext = rangeMapKeyInContext;
+        outRecDescs[0] = outputRecordDescriptor;
+    }
+
+    @Override
+    public void contributeActivities(IActivityGraphBuilder builder) {
+        ForwardDataActivity forwardDataActivity =
+                new ForwardDataActivity(new ActivityId(odId, FORWARD_DATA_ACTIVITY_ID));
+        RangeMapReaderActivity rangeMapReaderActivity =
+                new RangeMapReaderActivity(new ActivityId(odId, RANGEMAP_READER_ACTIVITY_ID));
+
+        // range map reader activity, its input is coming through the operator's in-port = 1 & activity's in-port = 0
+        builder.addActivity(this, rangeMapReaderActivity);
+        builder.addSourceEdge(1, rangeMapReaderActivity, 0);
+
+        // forward data activity, its input is coming through the operator's in-port = 0 & activity's in-port = 0
+        builder.addActivity(this, forwardDataActivity);
+        builder.addSourceEdge(0, forwardDataActivity, 0);
+
+        // forward data activity will wait for the range map reader activity
+        builder.addBlockingEdge(rangeMapReaderActivity, forwardDataActivity);
+
+        // data leaves from the operator's out-port = 0 & forward data activity's out-port = 0
+        builder.addTargetEdge(0, forwardDataActivity, 0);
+    }
+
+    /**
+     * Internal class that is used to transfer the {@link RangeMap} object between activities in different ctx but in
+     * the same NC, from {@link RangeMapReaderActivity} to {@link ForwardDataActivity}. These activities will share
+     * the {@link org.apache.hyracks.api.job.IOperatorEnvironment} of the {@link org.apache.hyracks.control.nc.Joblet}
+     * where the range map will be stored.
+     */
+    private class RangeMapState extends AbstractStateObject {
+        RangeMap rangeMap;
+
+        private RangeMapState(JobId jobId, TaskId stateObjectKey) {
+            super(jobId, stateObjectKey);
+        }
+    }
+
+    /**
+     * Range map reader activity. {@see {@link RangeMapReaderActivityNodePushable}}
+     */
+    private class RangeMapReaderActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private RangeMapReaderActivity(ActivityId activityId) {
+            super(activityId);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+                throws HyracksDataException {
+            RecordDescriptor inputRecordDescriptor = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
+            return new RangeMapReaderActivityNodePushable(ctx, inputRecordDescriptor, getActivityId(), partition);
+        }
+    }
+
+    /**
+     * Forward data activity. {@see {@link ForwardDataActivityNodePushable}}
+     */
+    private class ForwardDataActivity extends AbstractActivityNode {
+        private static final long serialVersionUID = 1L;
+
+        private ForwardDataActivity(ActivityId activityId) {
+            super(activityId);
+        }
+
+        @Override
+        public IOperatorNodePushable createPushRuntime(final IHyracksTaskContext ctx,
+                IRecordDescriptorProvider recordDescProvider, final int partition, int nPartitions)
+                throws HyracksDataException {
+            return new ForwardDataActivityNodePushable(ctx, partition);
+        }
+    }
+
+    private class RangeMapReaderActivityNodePushable extends AbstractUnaryInputSinkOperatorNodePushable {
+        private final FrameTupleAccessor frameTupleAccessor;
+        private final FrameTupleReference frameTupleReference;
+        private final IHyracksTaskContext ctx;
+        private final ActivityId activityId;
+        private final int partition;
+        private int numFields;
+        private byte[] splitValues;
+        private int[] splitValuesEndOffsets;
+
+        private RangeMapReaderActivityNodePushable(IHyracksTaskContext ctx, RecordDescriptor inputRecordDescriptor,
+                ActivityId activityId, int partition) {
+            this.ctx = ctx;
+            this.frameTupleAccessor = new FrameTupleAccessor(inputRecordDescriptor);
+            this.frameTupleReference = new FrameTupleReference();
+            this.activityId = activityId;
+            this.partition = partition;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            // this activity does not have a consumer to open (it's a sink), and nothing to initialize
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            // "buffer" contains the serialized range map sent by a range map computer function.
+            // deserialize the range map
+            frameTupleAccessor.reset(buffer);
+            if (frameTupleAccessor.getTupleCount() != 1) {
+                throw HyracksDataException.create(ErrorCode.ONE_TUPLE_RANGEMAP_EXPECTED, sourceLoc);
+            }
+            frameTupleReference.reset(frameTupleAccessor, 0);
+            byte[] rangeMap = frameTupleReference.getFieldData(0);
+            int offset = frameTupleReference.getFieldStart(0);
+            int length = frameTupleReference.getFieldLength(0);
+
+            ByteArrayInputStream rangeMapIn = new ByteArrayInputStream(rangeMap, offset, length);
+            DataInputStream dataInputStream = new DataInputStream(rangeMapIn);
+            numFields = IntegerSerializerDeserializer.read(dataInputStream);
+            splitValues = ByteArraySerializerDeserializer.read(dataInputStream);
+            splitValuesEndOffsets = IntArraySerializerDeserializer.read(dataInputStream);
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            // it's a sink node pushable, nothing to fail
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            // expecting a range map
+            if (numFields <= 0 || splitValues == null || splitValuesEndOffsets == null) {
+                throw HyracksDataException.create(ErrorCode.NO_RANGEMAP_PRODUCED, sourceLoc);
+            }
+            // store the range map in the state object of ctx so that next activity (forward) could retrieve it
+            TaskId rangeMapReaderTaskId = new TaskId(activityId, partition);
+            RangeMapState rangeMapState = new RangeMapState(ctx.getJobletContext().getJobId(), rangeMapReaderTaskId);
+            rangeMapState.rangeMap = new RangeMap(numFields, splitValues, splitValuesEndOffsets);
+            ctx.setStateObject(rangeMapState);
+        }
+    }
+
+    private class ForwardDataActivityNodePushable extends AbstractUnaryInputUnaryOutputOperatorNodePushable {
+        private final IHyracksTaskContext ctx;
+        private final int partition;
+
+        /**
+         * @param ctx used to retrieve the range map stored by the range reader activity.
+         * @param partition used to create the same task id used by the range reader activity for storing the range.
+         */
+        private ForwardDataActivityNodePushable(IHyracksTaskContext ctx, int partition) {
+            this.ctx = ctx;
+            this.partition = partition;
+        }
+
+        @Override
+        public void open() throws HyracksDataException {
+            // retrieve the range map from the state object (previous activity should have already stored it)
+            // then deposit it into the ctx so that MToN-partition can pick it up
+            Object stateObjKey = new TaskId(new ActivityId(odId, RANGEMAP_READER_ACTIVITY_ID), partition);
+            RangeMapState rangeMapState = (RangeMapState) ctx.getStateObject(stateObjKey);
+            TaskUtil.put(rangeMapKeyInContext, rangeMapState.rangeMap, ctx);
+            writer.open();
+        }
+
+        @Override
+        public void nextFrame(ByteBuffer buffer) throws HyracksDataException {
+            writer.nextFrame(buffer);
+        }
+
+        @Override
+        public void fail() throws HyracksDataException {
+            writer.fail();
+        }
+
+        @Override
+        public void close() throws HyracksDataException {
+            writer.close();
+        }
+
+        @Override
+        public void flush() throws HyracksDataException {
+            writer.flush();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
index 4a77b3c..0bead97 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/sort/AbstractExternalSortRunMerger.java
@@ -111,7 +111,7 @@ public abstract class AbstractExternalSortRunMerger {
 
                 int stop = runs.size();
                 currentGenerationRunAvailable.set(0, stop);
-
+                int numberOfPasses = 1;
                 while (true) {
 
                     int unUsed = selectPartialRuns(maxMergeWidth * ctx.getInitialFrameSize(), runs, partialRuns,
@@ -147,7 +147,7 @@ public abstract class AbstractExternalSortRunMerger {
                         runs.add(reader);
 
                         if (currentGenerationRunAvailable.isEmpty()) {
-
+                            numberOfPasses++;
                             if (LOGGER.isDebugEnabled()) {
                                 LOGGER.debug("generated runs:" + stop);
                             }
@@ -157,7 +157,10 @@ public abstract class AbstractExternalSortRunMerger {
                             stop = runs.size();
                         }
                     } else {
-                        LOGGER.debug("final runs: {}", stop);
+                        if (LOGGER.isDebugEnabled()) {
+                            LOGGER.debug("final runs: {}", stop);
+                            LOGGER.debug("number of passes: " + numberOfPasses);
+                        }
                         merge(finalWriter, partialRuns);
                         break;
                     }


[02/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
index 3335d71..fa11f73 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/ExtractCommonOperatorsRule.java
@@ -52,6 +52,10 @@ import org.apache.hyracks.algebricks.core.rewriter.base.HeuristicOptimizer;
 import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.api.exceptions.SourceLocation;
 
+/**
+ * Pre-conditions:
+ *      FixReplicateOperatorOutputsRule should be fired
+ */
 public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
 
     private final HashMap<Mutable<ILogicalOperator>, List<Mutable<ILogicalOperator>>> childrenToParents =
@@ -62,6 +66,8 @@ public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
     private final HashMap<Mutable<ILogicalOperator>, MutableInt> clusterMap = new HashMap<>();
     private final HashMap<Integer, BitSet> clusterWaitForMap = new HashMap<>();
     private int lastUsedClusterId = 0;
+    private final Map<Mutable<ILogicalOperator>, BitSet> replicateToOutputs = new HashMap<>();
+    private final List<Pair<Mutable<ILogicalOperator>, Boolean>> newOutputs = new ArrayList<>();
 
     @Override
     public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -268,11 +274,71 @@ public class ExtractCommonOperatorsRule implements IAlgebraicRewriteRule {
                     context.computeAndSetTypeEnvironmentForOperator(parentOp);
                 }
             }
+            cleanupPlan();
             rewritten = true;
         }
         return rewritten;
     }
 
+    /**
+     * Cleans up the plan after combining similar branches into one branch making sure parents & children point to
+     * each other correctly.
+     */
+    private void cleanupPlan() {
+        for (Mutable<ILogicalOperator> root : roots) {
+            replicateToOutputs.clear();
+            newOutputs.clear();
+            findReplicateOp(root, replicateToOutputs);
+            cleanup(replicateToOutputs, newOutputs);
+        }
+    }
+
+    /**
+     * Updates the outputs references of a replicate operator to points to the valid parents.
+     * @param replicateToOutputs where the replicate operators are stored with its valid parents.
+     * @param newOutputs the valid parents of replicate operator.
+     */
+    private void cleanup(Map<Mutable<ILogicalOperator>, BitSet> replicateToOutputs,
+            List<Pair<Mutable<ILogicalOperator>, Boolean>> newOutputs) {
+        replicateToOutputs.forEach((repRef, allOutputs) -> {
+            newOutputs.clear();
+            // get the indexes that are set in the BitSet
+            allOutputs.stream().forEach(outIndex -> {
+                newOutputs.add(new Pair<>(((AbstractReplicateOperator) repRef.getValue()).getOutputs().get(outIndex),
+                        ((AbstractReplicateOperator) repRef.getValue()).getOutputMaterializationFlags()[outIndex]));
+            });
+            ((AbstractReplicateOperator) repRef.getValue()).setOutputs(newOutputs);
+        });
+    }
+
+    /**
+     * Collects all replicate operator starting from {@param parent} and all its descendants and keeps track of the
+     * valid parents of a replicate operator. The indexes of valid parents will be set in the BitSet.
+     * @param parent the current operator in consideration for which we want to find replicate op children.
+     * @param replicateToOutputs where the replicate operators will be stored with all its parents (valid & invalid).
+     */
+    private void findReplicateOp(Mutable<ILogicalOperator> parent,
+            Map<Mutable<ILogicalOperator>, BitSet> replicateToOutputs) {
+        List<Mutable<ILogicalOperator>> children = parent.getValue().getInputs();
+        for (Mutable<ILogicalOperator> childRef : children) {
+            AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
+            if (child.getOperatorTag() == LogicalOperatorTag.REPLICATE
+                    || child.getOperatorTag() == LogicalOperatorTag.SPLIT) {
+                AbstractReplicateOperator replicateChild = (AbstractReplicateOperator) child;
+                int parentIndex = replicateChild.getOutputs().indexOf(parent);
+                if (parentIndex >= 0) {
+                    BitSet replicateValidOutputs = replicateToOutputs.get(childRef);
+                    if (replicateValidOutputs == null) {
+                        replicateValidOutputs = new BitSet();
+                        replicateToOutputs.put(childRef, replicateValidOutputs);
+                    }
+                    replicateValidOutputs.set(parentIndex);
+                }
+            }
+            findReplicateOp(childRef, replicateToOutputs);
+        }
+    }
+
     private void genCandidates(IOptimizationContext context) throws AlgebricksException {
         List<List<Mutable<ILogicalOperator>>> previousEquivalenceClasses =
                 new ArrayList<List<Mutable<ILogicalOperator>>>();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/InlineVariablesRule.java
----------------------------------------------------------------------
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 6967271..5d6237a 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
@@ -67,15 +67,18 @@ import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
  */
 public class InlineVariablesRule implements IAlgebraicRewriteRule {
 
-    // Map of variables that could be replaced by their producing expression.
-    // Populated during the top-down sweep of the plan.
-    protected Map<LogicalVariable, ILogicalExpression> varAssignRhs = new HashMap<>();
-    // Visitor for replacing variable reference expressions with their originating expression.
+    // map of variables that could be replaced by their producing expression.
+    // populated during the top-down sweep of the plan.
+    private Map<LogicalVariable, ILogicalExpression> varAssignRhs = new HashMap<>();
+    // visitor for replacing variable reference expressions with their originating expression.
     protected InlineVariablesVisitor inlineVisitor = new InlineVariablesVisitor(varAssignRhs);
-    // Set of FunctionIdentifiers that we should not inline.
+    // set of FunctionIdentifiers that we should not inline.
     protected Set<FunctionIdentifier> doNotInlineFuncs = new HashSet<>();
-    // Indicates whether the rule has been run
-    protected boolean hasRun = false;
+    // indicates whether the rule has been run
+    private boolean hasRun = false;
+    // set to prevent re-visiting a subtree from the other sides. Operators with multiple outputs are the ones that
+    // could be re-visited twice or more (e.g. replicate and split operators)
+    private final Map<ILogicalOperator, Boolean> subTreesDone = new HashMap<>();
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context) {
@@ -103,6 +106,7 @@ public class InlineVariablesRule implements IAlgebraicRewriteRule {
     protected void prepare(IOptimizationContext context) {
         varAssignRhs.clear();
         inlineVisitor.setContext(context);
+        subTreesDone.clear();
     }
 
     protected boolean performBottomUpAction(AbstractLogicalOperator op) throws AlgebricksException {
@@ -118,10 +122,14 @@ public class InlineVariablesRule implements IAlgebraicRewriteRule {
         return false;
     }
 
-    protected boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+    private boolean inlineVariables(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
             throws AlgebricksException {
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
 
+        // check if you have already visited the subtree rooted at this operator
+        if (subTreesDone.containsKey(op)) {
+            return subTreesDone.get(op);
+        }
         // Update mapping from variables to expressions during top-down traversal.
         if (op.getOperatorTag() == LogicalOperatorTag.ASSIGN) {
             AssignOperator assignOp = (AssignOperator) op;
@@ -183,6 +191,10 @@ public class InlineVariablesRule implements IAlgebraicRewriteRule {
             // Re-enable rules that we may have already tried. They could be applicable now after inlining.
             context.removeFromAlreadyCompared(opRef.getValue());
         }
+        // mark the subtree rooted at op as visited so that you don't visit it again
+        if (op.getOperatorTag() == LogicalOperatorTag.REPLICATE || op.getOperatorTag() == LogicalOperatorTag.SPLIT) {
+            subTreesDone.put(op, modified);
+        }
 
         return modified;
     }
@@ -209,7 +221,7 @@ public class InlineVariablesRule implements IAlgebraicRewriteRule {
             this.context = context;
         }
 
-        public void setOperator(ILogicalOperator op) throws AlgebricksException {
+        public void setOperator(ILogicalOperator op) {
             this.op = op;
             liveVars.clear();
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
index 4869761..4273553 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/SetAlgebricksPhysicalOperatorsRule.java
@@ -59,6 +59,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.physical.DataSourceS
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.DistributeResultPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.EmptyTupleSourcePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ForwardPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.IndexBulkloadPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.IndexInsertDeleteUpsertPOperator;
@@ -394,6 +395,9 @@ public class SetAlgebricksPhysicalOperatorsRule implements IAlgebraicRewriteRule
                     op.setPhysicalOperator(new SinkPOperator());
                     break;
                 }
+                case FORWARD:
+                    op.setPhysicalOperator(new ForwardPOperator());
+                    break;
             }
         }
         if (op.hasNestedPlans()) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
index 35aa984..6b09894 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/ReplaceNtsWithSubplanInputOperatorVisitor.java
@@ -34,10 +34,11 @@ import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 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.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -246,6 +247,11 @@ class ReplaceNtsWithSubplanInputOperatorVisitor implements IQueryOperatorVisitor
     }
 
     @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return visit(op);
+    }
+
+    @Override
     public ILogicalOperator visitIntersectOperator(IntersectOperator op, Void arg) throws AlgebricksException {
         return visit(op);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
index 0dcc83a..2068ef3 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputer.java
@@ -22,5 +22,23 @@ import org.apache.hyracks.api.comm.IFrameTupleAccessor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
 public interface ITuplePartitionComputer {
+    /**
+     * For the tuple (located at tIndex in the frame), it determines which target partition (0,1,... nParts-1) the tuple
+     * should be sent/written to.
+     * @param accessor The accessor of the frame to access tuples
+     * @param tIndex The index of the tuple in consideration
+     * @param nParts The number of target partitions
+     * @return The chosen target partition number as dictated by the logic of the partition computer
+     * @throws HyracksDataException
+     */
     public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException;
+
+    /**
+     * Gives the data partitioner a chance to set up its environment before it starts partitioning tuples. This method
+     * should be called in the open() of {@link org.apache.hyracks.api.comm.IFrameWriter}. The default implementation
+     * is "do nothing".
+     * @throws HyracksDataException
+     */
+    public default void initialize() throws HyracksDataException {
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
index cde0057..81f9053 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/dataflow/value/ITuplePartitionComputerFactory.java
@@ -20,6 +20,8 @@ package org.apache.hyracks.api.dataflow.value;
 
 import java.io.Serializable;
 
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+
 public interface ITuplePartitionComputerFactory extends Serializable {
-    public ITuplePartitionComputer createPartitioner();
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext);
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/exceptions/ErrorCode.java
----------------------------------------------------------------------
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 09193d9..7d126ac 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
@@ -149,6 +149,9 @@ public class ErrorCode {
     public static final int UNDEFINED_INVERTED_LIST_MERGE_TYPE = 113;
     public static final int NODE_IS_NOT_ACTIVE = 114;
     public static final int LOCAL_NETWORK_ERROR = 115;
+    public static final int ONE_TUPLE_RANGEMAP_EXPECTED = 116;
+    public static final int NO_RANGEMAP_PRODUCED = 117;
+    public static final int RANGEMAP_NOT_FOUND = 118;
 
     // Compilation error codes.
     public static final int RULECOLLECTION_NOT_INSTANCE_OF_LIST = 10000;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-api/src/main/resources/errormsg/en.properties
----------------------------------------------------------------------
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 c704d7e..50e92b3 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
@@ -132,6 +132,9 @@
 113 = Undefined inverted-list merge type: %1$s
 114 = Node (%1$s) is not active
 115 = Local network error
+116 = One tuple rangemap is expected
+117 = No range map produced for parallel sort
+118 = Range map was not found for parallel sort
 
 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

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleDataInputStream.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleDataInputStream.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleDataInputStream.java
new file mode 100644
index 0000000..d858a7f
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleDataInputStream.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.hyracks.data.std.util;
+
+import java.io.DataInputStream;
+
+public class ByteArrayAccessibleDataInputStream extends DataInputStream {
+
+    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
+        super(in);
+    }
+
+    public ByteArrayAccessibleInputStream getInputStream() {
+        return (ByteArrayAccessibleInputStream) in;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleInputStream.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleInputStream.java b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleInputStream.java
new file mode 100644
index 0000000..2785751
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-data/hyracks-data-std/src/main/java/org/apache/hyracks/data/std/util/ByteArrayAccessibleInputStream.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.hyracks.data.std.util;
+
+import java.io.ByteArrayInputStream;
+
+public class ByteArrayAccessibleInputStream extends ByteArrayInputStream {
+
+    public ByteArrayAccessibleInputStream(byte[] buf, int offset, int length) {
+        super(buf, offset, length);
+    }
+
+    public void setContent(byte[] buf, int offset, int length) {
+        this.buf = buf;
+        this.pos = offset;
+        this.count = Math.min(offset + length, buf.length);
+        this.mark = offset;
+    }
+
+    public byte[] getArray() {
+        return buf;
+    }
+
+    public int getPosition() {
+        return pos;
+    }
+
+    public int getCount() {
+        return count;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
index dc66d19..ab5ab01 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/FieldHashPartitionComputerFactory.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.dataflow.common.data.partition;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunction;
 import org.apache.hyracks.api.dataflow.value.IBinaryHashFunctionFactory;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
@@ -36,7 +37,7 @@ public class FieldHashPartitionComputerFactory implements ITuplePartitionCompute
     }
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         final IBinaryHashFunction[] hashFunctions = new IBinaryHashFunction[hashFunctionFactories.length];
         for (int i = 0; i < hashFunctionFactories.length; ++i) {
             hashFunctions[i] = hashFunctionFactories[i].createBinaryHashFunction();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/OnePartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/OnePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/OnePartitionComputerFactory.java
new file mode 100644
index 0000000..e55841a
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/OnePartitionComputerFactory.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.hyracks.dataflow.common.data.partition;
+
+import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+
+public class OnePartitionComputerFactory implements ITuplePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+
+    @Override
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
+        return new ITuplePartitionComputer() {
+            @Override
+            public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) {
+                return 0;
+            }
+        };
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
index e034af0..63d01fc 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RandomPartitionComputerFactory.java
@@ -21,6 +21,7 @@ package org.apache.hyracks.dataflow.common.data.partition;
 import java.util.Random;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -30,7 +31,7 @@ public class RandomPartitionComputerFactory implements ITuplePartitionComputerFa
     private static final long serialVersionUID = 1L;
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         return new ITuplePartitionComputer() {
 
             private final Random random = new Random();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
index 9cb11fa..1821d78 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/RepartitionComputerFactory.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.dataflow.common.data.partition;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -35,9 +36,9 @@ public class RepartitionComputerFactory implements ITuplePartitionComputerFactor
     }
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         return new ITuplePartitionComputer() {
-            private ITuplePartitionComputer delegate = delegateFactory.createPartitioner();
+            private ITuplePartitionComputer delegate = delegateFactory.createPartitioner(hyracksTaskContext);
 
             @Override
             public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/DynamicFieldRangePartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/DynamicFieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/DynamicFieldRangePartitionComputerFactory.java
new file mode 100644
index 0000000..bc642a9
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/DynamicFieldRangePartitionComputerFactory.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.hyracks.dataflow.common.data.partition.range;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.exceptions.ErrorCode;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.exceptions.SourceLocation;
+import org.apache.hyracks.dataflow.common.utils.TaskUtil;
+
+public class DynamicFieldRangePartitionComputerFactory extends FieldRangePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private final String rangeMapKeyInContext;
+    private final SourceLocation sourceLocation;
+
+    public DynamicFieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
+            String rangeMapKeyInContext, SourceLocation sourceLocation) {
+        super(rangeFields, comparatorFactories);
+        this.rangeMapKeyInContext = rangeMapKeyInContext;
+        this.sourceLocation = sourceLocation;
+    }
+
+    @Override
+    protected RangeMap getRangeMap(IHyracksTaskContext hyracksTaskContext) throws HyracksDataException {
+        RangeMap rangeMap = TaskUtil.get(rangeMapKeyInContext, hyracksTaskContext);
+        if (rangeMap == null) {
+            throw HyracksDataException.create(ErrorCode.RANGEMAP_NOT_FOUND, sourceLocation);
+        }
+        return rangeMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
index d58a248..55d4420 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/FieldRangePartitionComputerFactory.java
@@ -19,36 +19,41 @@
 package org.apache.hyracks.dataflow.common.data.partition.range;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+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.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 
-public class FieldRangePartitionComputerFactory implements ITuplePartitionComputerFactory {
+public abstract class FieldRangePartitionComputerFactory implements ITuplePartitionComputerFactory {
     private static final long serialVersionUID = 1L;
     private final int[] rangeFields;
-    private IRangeMap rangeMap;
     private IBinaryComparatorFactory[] comparatorFactories;
 
-    public FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
-            IRangeMap rangeMap) {
+    protected FieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories) {
         this.rangeFields = rangeFields;
         this.comparatorFactories = comparatorFactories;
-        this.rangeMap = rangeMap;
     }
 
+    protected abstract RangeMap getRangeMap(IHyracksTaskContext hyracksTaskContext) throws HyracksDataException;
+
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext hyracksTaskContext) {
         final IBinaryComparator[] comparators = new IBinaryComparator[comparatorFactories.length];
         for (int i = 0; i < comparatorFactories.length; ++i) {
             comparators[i] = comparatorFactories[i].createBinaryComparator();
         }
+
         return new ITuplePartitionComputer() {
+            private RangeMap rangeMap;
+
+            @Override
+            public void initialize() throws HyracksDataException {
+                rangeMap = getRangeMap(hyracksTaskContext);
+            }
+
             @Override
-            /**
-             * Determine the range partition.
-             */
             public int partition(IFrameTupleAccessor accessor, int tIndex, int nParts) throws HyracksDataException {
                 if (nParts == 1) {
                     return 0;
@@ -62,13 +67,10 @@ public class FieldRangePartitionComputerFactory implements ITuplePartitionComput
                 return (int) Math.floor(slotIndex / rangesPerPart);
             }
 
-            /*
-             * Determine the range partition.
-             */
-            public int getRangePartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
+            private int getRangePartition(IFrameTupleAccessor accessor, int tIndex) throws HyracksDataException {
                 int slotIndex = 0;
-                for (int i = 0; i < rangeMap.getSplitCount(); ++i) {
-                    int c = compareSlotAndFields(accessor, tIndex, i);
+                for (int slotNumber = 0; slotNumber < rangeMap.getSplitCount(); ++slotNumber) {
+                    int c = compareSlotAndFields(accessor, tIndex, slotNumber);
                     if (c < 0) {
                         return slotIndex;
                     }
@@ -77,18 +79,18 @@ public class FieldRangePartitionComputerFactory implements ITuplePartitionComput
                 return slotIndex;
             }
 
-            public int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int fieldIndex)
+            private int compareSlotAndFields(IFrameTupleAccessor accessor, int tIndex, int slotNumber)
                     throws HyracksDataException {
                 int c = 0;
                 int startOffset = accessor.getTupleStartOffset(tIndex);
                 int slotLength = accessor.getFieldSlotsLength();
-                for (int f = 0; f < comparators.length; ++f) {
-                    int fIdx = rangeFields[f];
+                for (int fieldNum = 0; fieldNum < comparators.length; ++fieldNum) {
+                    int fIdx = rangeFields[fieldNum];
                     int fStart = accessor.getFieldStartOffset(tIndex, fIdx);
                     int fEnd = accessor.getFieldEndOffset(tIndex, fIdx);
-                    c = comparators[f].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart,
-                            fEnd - fStart, rangeMap.getByteArray(fieldIndex, f), rangeMap.getStartOffset(fieldIndex, f),
-                            rangeMap.getLength(fieldIndex, f));
+                    c = comparators[fieldNum].compare(accessor.getBuffer().array(), startOffset + slotLength + fStart,
+                            fEnd - fStart, rangeMap.getByteArray(), rangeMap.getStartOffset(fieldNum, slotNumber),
+                            rangeMap.getLength(fieldNum, slotNumber));
                     if (c != 0) {
                         return c;
                     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java
deleted file mode 100644
index 5c5f34b..0000000
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/IRangeMap.java
+++ /dev/null
@@ -1,35 +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.hyracks.dataflow.common.data.partition.range;
-
-import org.apache.hyracks.data.std.api.IPointable;
-
-public interface IRangeMap {
-    public IPointable getFieldSplit(int columnIndex, int splitIndex);
-
-    public int getSplitCount();
-
-    public byte[] getByteArray(int columnIndex, int splitIndex);
-
-    public int getStartOffset(int columnIndex, int splitIndex);
-
-    public int getLength(int columnIndex, int splitIndex);
-
-    public int getTag(int columnIndex, int splitIndex);
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
index 98acbc0..714e3c0 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/RangeMap.java
@@ -19,80 +19,110 @@
 package org.apache.hyracks.dataflow.common.data.partition.range;
 
 import java.io.Serializable;
-
-import org.apache.hyracks.data.std.api.IPointable;
-import org.apache.hyracks.data.std.primitive.VoidPointable;
+import java.util.Arrays;
+import java.util.Objects;
 
 /**
- * The range map stores the field split values in an byte array.
- * The first split value for each field followed by the second split value for each field, etc.
+ * <pre>
+ * The range map stores the fields split values in a byte array.
+ * The first split value for each field followed by the second split value for each field, etc. For example:
+ *                  split_point_idx0    split_point_idx1    split_point_idx2    split_point_idx3    split_point_idx4
+ * in the byte[]:   f0,f1,f2            f0,f1,f2            f0,f1,f2            f0,f1,f2            f0,f1,f2
+ * numFields would be = 3
+ * we have 5 split points, which gives us 6 partitions:
+ *      p1  |       p2      |       p3      |       p4      |       p5      |       p6
+ *          sp0             sp1             sp2             sp3             sp4
+ * endOffsets.length would be = 15
+ * </pre>
  */
-public class RangeMap implements IRangeMap, Serializable {
-    private final int fields;
+public class RangeMap implements Serializable {
+    private final int numFields;
     private final byte[] bytes;
-    private final int[] offsets;
+    private final int[] endOffsets;
 
-    public RangeMap(int fields, byte[] bytes, int[] offsets) {
-        this.fields = fields;
+    public RangeMap(int numFields, byte[] bytes, int[] endOffsets) {
+        this.numFields = numFields;
         this.bytes = bytes;
-        this.offsets = offsets;
-    }
-
-    @Override
-    public IPointable getFieldSplit(int columnIndex, int splitIndex) {
-        IPointable p = VoidPointable.FACTORY.createPointable();
-        int index = getFieldIndex(columnIndex, splitIndex);
-        p.set(bytes, getFieldStart(index), getFieldLength(index));
-        return p;
+        this.endOffsets = endOffsets;
     }
 
-    @Override
     public int getSplitCount() {
-        return offsets.length / fields;
+        return endOffsets.length / numFields;
     }
 
-    @Override
-    public byte[] getByteArray(int columnIndex, int splitIndex) {
+    public byte[] getByteArray() {
         return bytes;
     }
 
-    @Override
-    public int getTag(int columnIndex, int splitIndex) {
-        return getFieldTag(getFieldIndex(columnIndex, splitIndex));
+    public int getTag(int fieldIndex, int splitIndex) {
+        return getSplitValueTag(getSplitValueIndex(fieldIndex, splitIndex));
     }
 
-    @Override
-    public int getStartOffset(int columnIndex, int splitIndex) {
-        return getFieldStart(getFieldIndex(columnIndex, splitIndex));
+    public int getStartOffset(int fieldIndex, int splitIndex) {
+        return getSplitValueStart(getSplitValueIndex(fieldIndex, splitIndex));
     }
 
-    @Override
-    public int getLength(int columnIndex, int splitIndex) {
-        return getFieldLength(getFieldIndex(columnIndex, splitIndex));
+    public int getLength(int fieldIndex, int splitIndex) {
+        return getSplitValueLength(getSplitValueIndex(fieldIndex, splitIndex));
     }
 
-    private int getFieldIndex(int columnIndex, int splitIndex) {
-        return splitIndex * fields + columnIndex;
+    /** Translates fieldIndex & splitIndex into an index which is used to find information about that split value.
+     * The combination of a fieldIndex & splitIndex uniquely identifies a split value of interest.
+     * @param fieldIndex the field index within the splitIndex of interest (0 <= fieldIndex < numFields)
+     * @param splitIndex starts with 0,1,2,.. etc
+     * @return the index of the desired split value that could be used with {@code bytes} & {@code endOffsets}.
+     */
+    private int getSplitValueIndex(int fieldIndex, int splitIndex) {
+        return splitIndex * numFields + fieldIndex;
     }
 
-    private int getFieldTag(int index) {
-        return bytes[getFieldStart(index)];
+    /**
+     * @param splitValueIndex is the combination of the split index + the field index within that split index
+     * @return the type tag of a specific field in a specific split point
+     */
+    private int getSplitValueTag(int splitValueIndex) {
+        return bytes[getSplitValueStart(splitValueIndex)];
     }
 
-    private int getFieldStart(int index) {
+    /**
+     * @param splitValueIndex is the combination of the split index + the field index within that split index
+     * @return the location of a split value in the byte array {@code bytes}
+     */
+    private int getSplitValueStart(int splitValueIndex) {
         int start = 0;
-        if (index != 0) {
-            start = offsets[index - 1];
+        if (splitValueIndex != 0) {
+            start = endOffsets[splitValueIndex - 1];
         }
         return start;
     }
 
-    private int getFieldLength(int index) {
-        int length = offsets[index];
-        if (index != 0) {
-            length -= offsets[index - 1];
+    /**
+     * @param splitValueIndex is the combination of the split index + the field index within that split index
+     * @return the length of a split value
+     */
+    private int getSplitValueLength(int splitValueIndex) {
+        int length = endOffsets[splitValueIndex];
+        if (splitValueIndex != 0) {
+            length -= endOffsets[splitValueIndex - 1];
         }
         return length;
     }
 
+    @Override
+    public int hashCode() {
+        return numFields + Arrays.hashCode(bytes) + Arrays.hashCode(endOffsets);
+    }
+
+    @Override
+    public boolean equals(Object object) {
+        if (this == object) {
+            return true;
+        }
+        if (!(object instanceof RangeMap)) {
+            return false;
+        }
+        RangeMap other = (RangeMap) object;
+        return numFields == other.numFields && Arrays.equals(endOffsets, other.endOffsets)
+                && Arrays.equals(bytes, other.bytes);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/StaticFieldRangePartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/StaticFieldRangePartitionComputerFactory.java b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/StaticFieldRangePartitionComputerFactory.java
new file mode 100644
index 0000000..b17c550
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-common/src/main/java/org/apache/hyracks/dataflow/common/data/partition/range/StaticFieldRangePartitionComputerFactory.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.hyracks.dataflow.common.data.partition.range;
+
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+
+public class StaticFieldRangePartitionComputerFactory extends FieldRangePartitionComputerFactory {
+    private static final long serialVersionUID = 1L;
+    private RangeMap rangeMap;
+
+    public StaticFieldRangePartitionComputerFactory(int[] rangeFields, IBinaryComparatorFactory[] comparatorFactories,
+            RangeMap rangeMap) {
+        super(rangeFields, comparatorFactories);
+        this.rangeMap = rangeMap;
+    }
+
+    @Override
+    protected RangeMap getRangeMap(IHyracksTaskContext hyracksTaskContext) {
+        return rangeMap;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
index 7d97507..8b57b15 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractMToNConnectorDescriptor.java
@@ -20,7 +20,14 @@ package org.apache.hyracks.dataflow.std.base;
 
 import java.util.BitSet;
 
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
+import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
 
 public abstract class AbstractMToNConnectorDescriptor extends AbstractConnectorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -47,4 +54,15 @@ public abstract class AbstractMToNConnectorDescriptor extends AbstractConnectorD
     public boolean allProducersToAllConsumers() {
         return true;
     }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        BitSet expectedPartitions = new BitSet(nProducerPartitions);
+        expectedPartitions.set(0, nProducerPartitions);
+        NonDeterministicChannelReader channelReader =
+                new NonDeterministicChannelReader(nProducerPartitions, expectedPartitions);
+        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
index ab553f6..4c728ce 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/base/AbstractReplicateOperatorDescriptor.java
@@ -180,6 +180,7 @@ public abstract class AbstractReplicateOperatorDescriptor extends AbstractOperat
 
                 @Override
                 public void fail() throws HyracksDataException {
+                    // TODO: shouldn't we fail the MaterializerTaskState state?
                     HyracksDataException hde = null;
                     for (int i = 0; i < numberOfNonMaterializedOutputs; i++) {
                         if (isOpen[i]) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/DeterministicPartitionBatchManager.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/DeterministicPartitionBatchManager.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/DeterministicPartitionBatchManager.java
new file mode 100644
index 0000000..c437619
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/DeterministicPartitionBatchManager.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.hyracks.dataflow.std.collectors;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.hyracks.api.channels.IInputChannel;
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.partitions.PartitionId;
+
+public class DeterministicPartitionBatchManager implements IPartitionBatchManager {
+    private final IFrameReader[] partitions;
+    private List<IFrameReader> partitionsList;
+
+    public DeterministicPartitionBatchManager(int nSenders) {
+        this.partitions = new IFrameReader[nSenders];
+    }
+
+    @Override
+    public synchronized void addPartition(PartitionId partitionId, IInputChannel channel) {
+        InputChannelFrameReader channelReader = new InputChannelFrameReader(channel);
+        channel.registerMonitor(channelReader);
+        partitions[partitionId.getSenderIndex()] = channelReader;
+        if (allPartitionsAdded()) {
+            partitionsList = new ArrayList<>(Arrays.asList(partitions));
+            notifyAll();
+        }
+    }
+
+    @Override
+    public synchronized void getNextBatch(List<IFrameReader> batch, int requestedSize) throws HyracksDataException {
+        while (!allPartitionsAdded()) {
+            try {
+                wait();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw HyracksDataException.create(e);
+            }
+        }
+        if (partitionsList.isEmpty()) {
+            return;
+        }
+        if (requestedSize >= partitionsList.size()) {
+            batch.addAll(partitionsList);
+            partitionsList.clear();
+        } else {
+            List<IFrameReader> subBatch = partitionsList.subList(0, requestedSize);
+            batch.addAll(subBatch);
+            subBatch.clear();
+        }
+    }
+
+    private synchronized boolean allPartitionsAdded() {
+        for (int i = 0; i < partitions.length; i++) {
+            if (partitions[i] == null) {
+                return false;
+            }
+        }
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/SequentialMergeFrameReader.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/SequentialMergeFrameReader.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/SequentialMergeFrameReader.java
new file mode 100644
index 0000000..2646c94
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/collectors/SequentialMergeFrameReader.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.
+ */
+package org.apache.hyracks.dataflow.std.collectors;
+
+import java.util.LinkedList;
+
+import org.apache.hyracks.api.comm.IFrame;
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+
+public class SequentialMergeFrameReader implements IFrameReader {
+    private final int numSenders;
+    private final IPartitionBatchManager partitionBatchManager;
+    private final LinkedList<IFrameReader> senders;
+    private boolean isOpen;
+
+    public SequentialMergeFrameReader(int numSenders, IPartitionBatchManager partitionBatchManager) {
+        this.numSenders = numSenders;
+        this.partitionBatchManager = partitionBatchManager;
+        this.senders = new LinkedList<>();
+        this.isOpen = false;
+    }
+
+    @Override
+    public void open() throws HyracksDataException {
+        if (!isOpen) {
+            isOpen = true;
+            // get all the senders and open them one by one
+            partitionBatchManager.getNextBatch(senders, numSenders);
+            for (IFrameReader sender : senders) {
+                sender.open();
+            }
+        }
+    }
+
+    @Override
+    public boolean nextFrame(IFrame outFrame) throws HyracksDataException {
+        IFrameReader currentSender;
+        while (!senders.isEmpty()) {
+            // process the sender at the beginning of the sequence
+            currentSender = senders.getFirst();
+            outFrame.reset();
+            if (currentSender.nextFrame(outFrame)) {
+                return true;
+            } else {
+                // done with the current sender, close it, remove it from the Q and process the next one in sequence
+                currentSender.close();
+                senders.removeFirst();
+            }
+        }
+        // done with all senders
+        return false;
+    }
+
+    @Override
+    public void close() throws HyracksDataException {
+        for (IFrameReader sender : senders) {
+            sender.close();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
index 920fdb8..0b6e40e 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/LocalityAwareMToNPartitioningConnectorDescriptor.java
@@ -61,7 +61,7 @@ public class LocalityAwareMToNPartitioningConnectorDescriptor extends AbstractMT
     public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
-        return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(),
+        return new LocalityAwarePartitionDataWriter(ctx, edwFactory, recordDesc, tpcf.createPartitioner(ctx),
                 nConsumerPartitions, localityMap, index);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
index 092b5f1..32618ee 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNBroadcastConnectorDescriptor.java
@@ -19,19 +19,14 @@
 package org.apache.hyracks.dataflow.std.connectors;
 
 import java.nio.ByteBuffer;
-import java.util.BitSet;
 
 import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.IPartitionCollector;
 import org.apache.hyracks.api.comm.IPartitionWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
-import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
 
 public class MToNBroadcastConnectorDescriptor extends AbstractMToNConnectorDescriptor {
 
@@ -123,15 +118,4 @@ public class MToNBroadcastConnectorDescriptor extends AbstractMToNConnectorDescr
             }
         };
     }
-
-    @Override
-    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
-            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
-        BitSet expectedPartitions = new BitSet(nProducerPartitions);
-        expectedPartitions.set(0, nProducerPartitions);
-        NonDeterministicChannelReader channelReader =
-                new NonDeterministicChannelReader(nProducerPartitions, expectedPartitions);
-        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
-        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
-    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
index 02fbedb..c11c08c 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningConnectorDescriptor.java
@@ -18,10 +18,7 @@
  */
 package org.apache.hyracks.dataflow.std.connectors;
 
-import java.util.BitSet;
-
 import org.apache.hyracks.api.comm.IFrameWriter;
-import org.apache.hyracks.api.comm.IPartitionCollector;
 import org.apache.hyracks.api.comm.IPartitionWriterFactory;
 import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
@@ -29,9 +26,6 @@ import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
 import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicChannelReader;
-import org.apache.hyracks.dataflow.std.collectors.NonDeterministicFrameReader;
-import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
 
 public class MToNPartitioningConnectorDescriptor extends AbstractMToNConnectorDescriptor {
     private static final long serialVersionUID = 1L;
@@ -46,18 +40,7 @@ public class MToNPartitioningConnectorDescriptor extends AbstractMToNConnectorDe
     public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
-        return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner());
-    }
-
-    @Override
-    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
-            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
-        BitSet expectedPartitions = new BitSet(nProducerPartitions);
-        expectedPartitions.set(0, nProducerPartitions);
-        NonDeterministicChannelReader channelReader =
-                new NonDeterministicChannelReader(nProducerPartitions, expectedPartitions);
-        NonDeterministicFrameReader frameReader = new NonDeterministicFrameReader(channelReader);
-        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, frameReader, channelReader);
+        return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner(ctx));
     }
 
     public ITuplePartitionComputerFactory getTuplePartitionComputerFactory() {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
index 026ca5e..e0ec5d6 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningMergingConnectorDescriptor.java
@@ -70,7 +70,7 @@ public class MToNPartitioningMergingConnectorDescriptor extends AbstractMToNConn
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         final PartitionDataWriter hashWriter =
-                new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner());
+                new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner(ctx));
         return hashWriter;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java
index f6996f1..af3ce06 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToNPartitioningWithMessageConnectorDescriptor.java
@@ -45,6 +45,6 @@ public class MToNPartitioningWithMessageConnectorDescriptor extends MToNPartitio
             IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
             throws HyracksDataException {
         return new PartitionWithMessageDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc,
-                tpcf.createPartitioner());
+                tpcf.createPartitioner(ctx));
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToOneSequentialMergingConnectorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToOneSequentialMergingConnectorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToOneSequentialMergingConnectorDescriptor.java
new file mode 100644
index 0000000..3decb69
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/MToOneSequentialMergingConnectorDescriptor.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.hyracks.dataflow.std.connectors;
+
+import java.util.BitSet;
+
+import org.apache.hyracks.api.comm.IFrameReader;
+import org.apache.hyracks.api.comm.IFrameWriter;
+import org.apache.hyracks.api.comm.IPartitionCollector;
+import org.apache.hyracks.api.comm.IPartitionWriterFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.OnePartitionComputerFactory;
+import org.apache.hyracks.dataflow.std.base.AbstractMToNConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.collectors.DeterministicPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.IPartitionBatchManager;
+import org.apache.hyracks.dataflow.std.collectors.PartitionCollector;
+import org.apache.hyracks.dataflow.std.collectors.SequentialMergeFrameReader;
+
+public class MToOneSequentialMergingConnectorDescriptor extends AbstractMToNConnectorDescriptor {
+    private static final long serialVersionUID = 1L;
+    private final ITuplePartitionComputerFactory tpcf;
+
+    public MToOneSequentialMergingConnectorDescriptor(IConnectorDescriptorRegistry spec) {
+        super(spec);
+        tpcf = new OnePartitionComputerFactory();
+    }
+
+    @Override
+    public IFrameWriter createPartitioner(IHyracksTaskContext ctx, RecordDescriptor recordDesc,
+            IPartitionWriterFactory edwFactory, int index, int nProducerPartitions, int nConsumerPartitions)
+            throws HyracksDataException {
+        // TODO(ali): create a single partition data writer instead
+        return new PartitionDataWriter(ctx, nConsumerPartitions, edwFactory, recordDesc, tpcf.createPartitioner(ctx));
+    }
+
+    @Override
+    public IPartitionCollector createPartitionCollector(IHyracksTaskContext ctx, RecordDescriptor recordDesc, int index,
+            int nProducerPartitions, int nConsumerPartitions) throws HyracksDataException {
+        IPartitionBatchManager pbm = new DeterministicPartitionBatchManager(nProducerPartitions);
+        IFrameReader sequentialMergeReader = new SequentialMergeFrameReader(nProducerPartitions, pbm);
+        BitSet expectedPartitions = new BitSet();
+        expectedPartitions.set(0, nProducerPartitions);
+        return new PartitionCollector(ctx, getConnectorId(), index, expectedPartitions, sequentialMergeReader, pbm);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
index 5e33275..d06d5d3 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/connectors/PartitionDataWriter.java
@@ -113,6 +113,7 @@ public class PartitionDataWriter implements IFrameWriter {
 
     @Override
     public void open() throws HyracksDataException {
+        tpc.initialize();
         for (int i = 0; i < pWriters.length; ++i) {
             isOpen[i] = true;
             pWriters[i].open();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
index dc250e6..034b054 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/HybridHashJoinOperatorDescriptor.java
@@ -190,7 +190,7 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                         ctx.getJobletContext().getJobId(), new TaskId(getActivityId(), partition));
                 private final FrameTupleAccessor accessorBuild = new FrameTupleAccessor(rd1);
                 private final ITuplePartitionComputer hpcBuild =
-                        new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner();
+                        new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner(ctx);
                 private final FrameTupleAppender appender = new FrameTupleAppender();
                 private final FrameTupleAppender ftappender = new FrameTupleAppender();
                 private IFrame[] bufferForPartitions;
@@ -303,9 +303,9 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                     }
 
                     ITuplePartitionComputer hpc0 =
-                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner(ctx);
                     ITuplePartitionComputer hpc1 =
-                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner(ctx);
                     int tableSize = (int) (state.memoryForHashtable * recordsPerFrame * factor);
                     ISerializableTable table = new SimpleSerializableHashTable(tableSize, ctx);
                     state.joiner =
@@ -385,7 +385,7 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                         new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories);
                 private final ITuplePartitionComputerFactory hpcf1 =
                         new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories);
-                private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner();
+                private final ITuplePartitionComputer hpcProbe = hpcf0.createPartitioner(ctx);
 
                 private final FrameTupleAppender appender = new FrameTupleAppender();
                 private final FrameTupleAppender ftap = new FrameTupleAppender();
@@ -476,9 +476,9 @@ public class HybridHashJoinOperatorDescriptor extends AbstractOperatorDescriptor
                             state.joiner.releaseMemory();
                         }
                         ITuplePartitionComputer hpcRep0 =
-                                new RepartitionComputerFactory(state.nPartitions, hpcf0).createPartitioner();
+                                new RepartitionComputerFactory(state.nPartitions, hpcf0).createPartitioner(ctx);
                         ITuplePartitionComputer hpcRep1 =
-                                new RepartitionComputerFactory(state.nPartitions, hpcf1).createPartitioner();
+                                new RepartitionComputerFactory(state.nPartitions, hpcf1).createPartitioner(ctx);
                         if (state.memoryForHashtable != memsize - 2) {
                             for (int i = 0; i < state.nPartitions; i++) {
                                 ByteBuffer buf = bufferForPartitions[i].getBuffer();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
index 3873bae..a5c17f2 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/join/InMemoryHashJoinOperatorDescriptor.java
@@ -182,9 +182,9 @@ public class InMemoryHashJoinOperatorDescriptor extends AbstractOperatorDescript
                 @Override
                 public void open() throws HyracksDataException {
                     ITuplePartitionComputer hpc0 =
-                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys0, hashFunctionFactories).createPartitioner(ctx);
                     ITuplePartitionComputer hpc1 =
-                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner();
+                            new FieldHashPartitionComputerFactory(keys1, hashFunctionFactories).createPartitioner(ctx);
                     state = new HashBuildTaskState(ctx.getJobletContext().getJobId(),
                             new TaskId(getActivityId(), partition));
                     ISerializableTable table = new SerializableHashTable(tableSize, ctx, bufferManager);


[18/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.sqlpp
new file mode 100644
index 0000000..a82fe5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-08_ps.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.
+ */
+/*
+ * Description  : Index join because there's a hint and the probe type is known (string). p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
new file mode 100644
index 0000000..759ce3a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-09_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Index join because there's a hint and the probe type is known (bigint). p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-08.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_bigint(t1.c_i64) /*+ indexnl */ = t2.c_i64
+order by t1.c_x, t2.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
new file mode 100644
index 0000000..1b585b6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+/*
+ *  Description     : BTree non-enforced index verification test
+ *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+ *  Expected Result : Success
+ *  Date            : 12th Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-04.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_s on TestOpen(c_s:string);
+
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_s = 'world'
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
new file mode 100644
index 0000000..b14b9d6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-05.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i64 on TestOpen(c_i64:int64);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i64 = 2
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
new file mode 100644
index 0000000..570c37e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-06.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i64 on TestOpen(c_i64:int64);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i64 > 2
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
new file mode 100644
index 0000000..ffb1cf8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-07.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i64 on TestOpen(c_i64:int64);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i64 > 2.0
+order by t.c_x;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
new file mode 100644
index 0000000..6de6121
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-08.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i8 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
new file mode 100644
index 0000000..3f1a7847
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-09.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i8 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2.5
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
new file mode 100644
index 0000000..6049e5f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.sqlpp
@@ -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.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *                  : when multiple indexes match
+  *  Expected Result : Success
+  *  Date            : 29 Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-104.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_3 on TestOpen(c_i64:int64);
+create index idx_4 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 2 and t.c_i64 < 3
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
new file mode 100644
index 0000000..afb08ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-10.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_d on TestOpen(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_d >= 3.25
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
new file mode 100644
index 0000000..b921fa3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.sqlpp
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+ /*
+  *  Description     : BTree non-enforced index verification test
+  *                  : Test that the secondary non-enforced BTree index is used. p_sort enabled.
+  *  Expected Result : Success
+  *  Date            : 12th Jun 2017
+  */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-index-non-enforced_btree-index-non-enforced-11.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen(TestOpenType) primary key c_id;
+create index idx_i8 on TestOpen(c_i8:int8);
+
+set `compiler.sort.parallel` "true";
+
+select t.c_x as res
+from TestOpen t
+where t.c_i8 > 499 and t.c_i8 < 99999
+order by t.c_x;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
new file mode 100644
index 0000000..254bd26
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orderby-desc-using-gby_ps.sqlpp
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 `gby-using-orderby-desc` if exists;
+create  dataverse `gby-using-orderby-desc`;
+
+use `gby-using-orderby-desc`;
+
+
+create type `gby-using-orderby-desc`.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type `gby-using-orderby-desc`.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  lastorder : {
+      oid : integer,
+      total : float
+  }
+
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Customers(CustomerType) primary key cid on group1;
+
+write output to asterix_nc1:"rttest/gby-using-orderby-desc.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'name':name,'age':age}
+from  Customers as c
+group by c.name as name, c.age as age
+order by name desc,age
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
new file mode 100644
index 0000000..7764086
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_01_ps.sqlpp
@@ -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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
new file mode 100644
index 0000000..3814e1c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive-open_02_ps.sqlpp
@@ -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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+{
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
new file mode 100644
index 0000000..80cec87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_01_ps.sqlpp
@@ -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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
new file mode 100644
index 0000000..de4604e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/orders-index-search-conjunctive_02_ps.sqlpp
@@ -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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create  nodegroup group1 if not exists  on
+    asterix_nc1,
+    asterix_nc2
+;
+create  dataset Orders(OrderType) primary key o_orderkey on group1;
+
+load  dataset Orders using localfs ((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+create  index idx_Orders_Custkey  on Orders (o_custkey) type btree;
+
+write output to asterix_nc1:"/tmp/index_search-conjunctive.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'o_orderkey':o.o_orderkey,'o_custkey':o.o_custkey,'o_totalprice':o.o_totalprice}
+from  Orders as o
+where ((o.o_custkey = 40) and (o.o_totalprice > 150000.0))
+order by o.o_orderkey
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.sqlpp
new file mode 100644
index 0000000..271565a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+* Description  : Test enabling parallel sort by setting the property
+* Expected Res : Success
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE CustomerType AS {
+  c_custkey : integer,
+  c_1 : integer
+};
+
+CREATE DATASET Customers(CustomerType) PRIMARY KEY c_custkey;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT VALUE c
+FROM Customers c
+ORDER BY c.c_1;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.sqlpp
new file mode 100644
index 0000000..e79e872
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.sqlpp
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+/*
+* Description  : Test disabling parallel sort by setting the property
+* Expected Res : Success
+*/
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE CustomerType AS {
+  c_custkey : integer,
+  c_1 : integer
+};
+
+CREATE DATASET Customers(CustomerType) PRIMARY KEY c_custkey;
+
+SET `compiler.sort.parallel` "false";
+
+SELECT VALUE c
+FROM Customers c
+ORDER BY c.c_1;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
new file mode 100644
index 0000000..3ee5774
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/q01_pricing_summary_report_nt_ps.sqlpp
@@ -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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
+
+write output to asterix_nc1:"rttest/tpch_q1_pricing_summary_report_nt.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'l_returnflag':l_returnflag,'l_linestatus':l_linestatus,'sum_qty':tpch.strict_sum((
+        select element i.l.l_quantity
+        from  g as i
+    )),'sum_base_price':tpch.strict_sum((
+        select element i.l.l_extendedprice
+        from  g as i
+    )),'sum_disc_price':tpch.strict_sum((
+        select element (i.l.l_extendedprice * (1 - i.l.l_discount))
+        from  g as i
+    )),'sum_charge':tpch.strict_sum((
+        select element (i.l.l_extendedprice * (1 - i.l.l_discount) * (1 + i.l.l_tax))
+        from  g as i
+    )),'ave_qty':tpch.strict_avg((
+        select element i.l.l_quantity
+        from  g as i
+    )),'ave_price':tpch.strict_avg((
+        select element i.l.l_extendedprice
+        from  g as i
+    )),'ave_disc':tpch.strict_avg((
+        select element i.l.l_discount
+        from  g as i
+    )),'count_order':tpch.strict_count(g)}
+from  LineItem as l
+where (l.l_shipdate <= '1998-09-02')
+/* +hash */
+group by l.l_returnflag as l_returnflag,l.l_linestatus as l_linestatus group as g
+order by l_returnflag,l_linestatus
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806_ps.sqlpp
new file mode 100644
index 0000000..28e60df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-1806_ps.sqlpp
@@ -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.
+ */
+
+/*
+ * Description  : This test case is to verify the fix for issue 1806. p_sort enabled.
+ * https://issues.apache.org/jira/browse/ASTERIXDB-1806
+ * Expected Res : Non-index utilization Plan
+ */
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type LineItemType as closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
+
+SET `import-private-functions` "true";
+SET `compiler.sort.parallel` "true";
+
+SELECT  l_returnflag,
+        l_linestatus,
+        sum(l_quantity) AS sum_qty,
+        sum(l_extendedprice) AS sum_base_price,
+        sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price,
+        sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge,
+        avg(l_quantity) AS ave_qty,
+        avg(l_extendedprice) AS ave_price,
+        avg(l_discount) AS ave_disc,
+        count(1) AS count_order
+FROM  LineItem l
+WHERE inject_failure(l.l_shipdate <= '1998-09-02', l.l_orderkey=5988)
+GROUP BY l_returnflag, l_linestatus
+ORDER BY l_returnflag, l_linestatus;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354_ps.sqlpp
new file mode 100644
index 0000000..722afae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2354_ps.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.
+ */
+
+set `compiler.sort.parallel` "true";
+
+SELECT ds.DatasetName as v1
+FROM Metadata.`Dataset` ds
+WHERE ds.DatasetName LIKE "Orders%"
+UNION ALL
+SELECT ds.DatasetName v1, idx.DatasetName v2, idx.IndexName v3
+FROM Metadata.`Index` idx, Metadata.`Dataset` ds
+WHERE ds.DatasetName LIKE "Orders%" and idx.DatasetName LIKE "Orders%"
+ORDER BY v1, v2, v3;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408_ps.sqlpp
new file mode 100644
index 0000000..b542ec7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-2408_ps.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.
+ */
+
+/*
+ * Description  : This test case is to verify the fix for issue ASTERIXDB-2408. p_sort enabled.
+ * https://issues.apache.org/jira/browse/ASTERIXDB-2408
+ * Expected Res :
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type testType as open {
+  id: int32
+};
+
+create dataset customers(testType) primary key id;
+create dataset orders(testType) primary key id;
+
+set `compiler.sort.parallel` "true";
+
+SELECT c.name, (
+  SELECT VALUE o.id
+  FROM orders o
+  WHERE o.customer_id = c.id
+) AS orders
+FROM customers c
+ORDER BY array_count(orders);
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2_ps.sqlpp
new file mode 100644
index 0000000..3b78d29
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-2_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue810. p_sort enabled.
+ * https://code.google.com/p/asterixdb/issues/detail?id=810
+ * Expected Res : SUCCESS
+ * Date         : 16th Nov. 2014
+ */
+DROP  DATAVERSE tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS CLOSED {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT l_returnflag AS l_returnflag,
+       l_linestatus AS l_linestatus,
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
+FROM LineItem AS l
+/* +hash */
+GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
+GROUP AS g
+LET cheap = (
+      SELECT ELEMENT g.l
+      FROM g
+      WHERE g.l.l_discount > 0.05
+),
+expensive = (
+      SELECT ELEMENT g.l
+      FROM g
+      WHERE g.l.l_discount <= 0.05
+)
+ORDER BY l_returnflag,l_linestatus;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3_ps.sqlpp
new file mode 100644
index 0000000..26974a6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810-3_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue810. p_sort enabled.
+ * https://code.google.com/p/asterixdb/issues/detail?id=810
+ * Expected Res : SUCCESS
+ * Date         : 16th Nov. 2014
+ */
+DROP  DATAVERSE tpch IF EXISTS;
+CREATE DATAVERSE tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS CLOSED {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT l_returnflag AS l_returnflag,
+       l_linestatus AS l_linestatus,
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
+FROM LineItem AS l
+/* +hash */
+GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
+GROUP AS g
+LET cheap = (
+      SELECT ELEMENT m
+      FROM (FROM g SELECT VALUE l) AS m
+      WHERE m.l_discount > 0.05
+),
+expensive = (
+      SELECT ELEMENT g.l
+      FROM g
+      WHERE g.l.l_discount <= 0.05
+)
+ORDER BY l_returnflag,l_linestatus;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810_ps.sqlpp
new file mode 100644
index 0000000..dcddc12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-ASTERIXDB-810_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : This test case is to verify the fix for issue810. p_sort enabled.
+ * https://code.google.com/p/asterixdb/issues/detail?id=810
+ * Expected Res : SUCCESS
+ * Date         : 16th Nov. 2014
+ */
+DROP  DATAVERSE tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS CLOSED {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_orderkey,l_linenumber;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT l_returnflag AS l_returnflag,
+       l_linestatus AS l_linestatus,
+       strict_count(cheap) AS count_cheaps,
+       strict_count(expensive) AS count_expensives
+FROM LineItem AS l
+/* +hash */
+GROUP BY l.l_returnflag AS l_returnflag,l.l_linestatus AS l_linestatus
+GROUP AS g
+LET cheap = (
+      SELECT ELEMENT m
+      FROM (FROM g SELECT VALUE l) AS m
+      WHERE m.l_discount > 0.05
+),
+expensive = (
+      SELECT ELEMENT m
+      FROM (FROM g SELECT VALUE l) AS m
+      WHERE m.l_discount <= 0.05
+)
+ORDER BY l_returnflag,l_linestatus;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562_ps.sqlpp
new file mode 100644
index 0000000..53c79ef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/query-issue562_ps.sqlpp
@@ -0,0 +1,108 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 issue562
+ * https://code.google.com/p/asterixdb/issues/detail?id=562
+ * Expected Res : SUCCESS
+ * Date         : 15th Jan. 2015
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+
+create type tpch.LineItemType as
+ closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : integer,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : integer,
+  o_custkey : integer,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : integer,
+  o_comment : string
+};
+
+create type tpch.CustomerType as
+ closed {
+  c_custkey : integer,
+  c_name : string,
+  c_address : string,
+  c_nationkey : integer,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+set `compiler.sort.parallel` "true";
+
+declare function q22_customer_tmp() {
+(
+    select element {'c_acctbal':c.c_acctbal,'c_custkey':c.c_custkey,'cntrycode':phone_substr}
+    from  Customer as c
+    with  phone_substr as tpch.substring(c.c_phone,0,2)
+    where ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17'))
+)
+};
+with  avg as tpch.strict_avg((
+      select element c.c_acctbal
+      from  Customer as c
+      with  phone_substr as tpch.substring(c.c_phone,0,2)
+      where ((c.c_acctbal > 0.0) and ((phone_substr = '13') or (phone_substr = '31') or (phone_substr = '23') or (phone_substr = '29') or (phone_substr = '30') or (phone_substr = '18') or (phone_substr = '17')))
+  ))
+select element {'cntrycode':cntrycode,'numcust':tpch.strict_count(g),'totacctbal':tpch.strict_sum((
+        select element i.ct.c_acctbal
+        from  g as i
+    ))}
+from  tpch.q22_customer_tmp() as ct
+where (tpch.strict_count((
+    select element o
+    from  Orders as o
+    where (ct.c_custkey = o.o_custkey)
+)) = 0)
+group by ct.cntrycode as cntrycode group as g
+order by cntrycode
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..2a61fa5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..3e1c881
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.`sender-location`,n) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
new file mode 100644
index 0000000..c14a25e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/skip-index/skip-ngram-index_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Notice the query hint to avoid using any secondary index to evaluate the predicate in the where clause
+ * Expected Res : Success
+ * Date         : 21th December 2013
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where  /*+ skip-index */ test.contains(o.title,'Multimedia')
+order by o.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization_ps.sqlpp
new file mode 100644
index 0000000..71bcb4f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/split-materialization_ps.sqlpp
@@ -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.
+ */
+
+drop  dataverse TinySocial if exists;
+create  dataverse TinySocial;
+
+use TinySocial;
+
+
+create type TinySocial.EmploymentType as
+{
+  `organization-name` : string,
+  `start-date` : date,
+  `end-date` : date?
+};
+
+create type TinySocial.FacebookUserType as
+ closed {
+  id : integer,
+  alias : string,
+  name : string,
+  `user-since` : datetime,
+  `friend-ids` : {{integer}},
+  employment : [EmploymentType]
+};
+
+create  dataset FacebookUsers(FacebookUserType) primary key id;
+set `compiler.sort.parallel` "true";
+
+with  lonelyusers as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  ),
+      lonelyusers2 as (
+      select element d
+      from  FacebookUsers as d
+      where (TinySocial.strict_count(d.`friend-ids`) < 2)
+  )
+select element {'user1':{'id':l1.id,'name':l1.name},'user2':{'id':l2.id,'name':l2.name}}
+from  lonelyusers as l1,
+      lonelyusers2 as l2
+where (l1.id < l2.id)
+order by l1.id,l2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists_ps.sqlpp
new file mode 100644
index 0000000..7fb60be
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/exists_ps.sqlpp
@@ -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.
+ */
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE OrderType AS CLOSED {
+  o_orderkey: integer,
+  o_custkey: integer,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: integer,
+  o_comment: string
+};
+
+CREATE TYPE CustomerType AS CLOSED {
+  c_custkey: integer,
+  c_name: string,
+  c_address: string,
+  c_nationkey: integer,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+};
+
+CREATE EXTERNAL DATASET Customer(CustomerType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+CREATE EXTERNAL DATASET Orders(OrderType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+SET `compiler.sort.parallel` "true";
+
+WITH q22_customer_tmp AS
+(
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
+    FROM  Customer
+)
+,
+avg AS (
+        SELECT ELEMENT AVG(c_acctbal)
+        FROM  Customer
+        WHERE c_acctbal > 0.0
+)[0]
+SELECT  cntrycode, count(ct) AS numcust, SUM(c_acctbal) AS totacctbal
+FROM  q22_customer_tmp AS ct
+WHERE c_acctbal > avg
+      AND EXISTS (SELECT * FROM Orders o WHERE o.o_custkey = ct.c_custkey)
+GROUP BY cntrycode
+ORDER BY cntrycode;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1_ps.sqlpp
new file mode 100644
index 0000000..308a618
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_1_ps.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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create dataset Customer(CustomerType) primary key c_custkey;
+
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2_ps.sqlpp
new file mode 100644
index 0000000..297116a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_2_ps.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.
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in ["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+order by c.c_custkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3_ps.sqlpp
new file mode 100644
index 0000000..d0d190d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_3_ps.sqlpp
@@ -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.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4_ps.sqlpp
new file mode 100644
index 0000000..2f40400
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_4_ps.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.
+ */
+
+// requesttype=application/json
+
+// param $p_arr:json=["Customer#000000001", "Customer#000000002", "Customer#000000003"]
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in $p_arr
+order by c.c_custkey;
\ No newline at end of file


[10/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8451f5b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810_ps.plan
@@ -0,0 +1,83 @@
+-- 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|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$120(ASC), $$121(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$120, $$121]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$108, $$109]  |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 [$$108(ASC), $$109(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |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[$$108, $$109]  |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 [$$108(ASC), $$109(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..b988a0a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue562_ps.plan
@@ -0,0 +1,135 @@
+-- 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[$$142]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$142]  |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[$$139]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$135]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- STREAM_SELECT  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$135(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$122][$$129]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$phone_substr][$$124]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- UNNEST  |UNPARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN  |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[$$142]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$142]  |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[$$139]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- HASH_PARTITION_EXCHANGE [$$139]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$135]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$135(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$122][$$129]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- HYBRID_HASH_JOIN [$$phone_substr][$$124]  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                  -- UNNEST  |UNPARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$129]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
index 6a471bc..07aed2b 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/issue730-index-only.plan
@@ -32,12 +32,11 @@
                                                     -- ASSIGN  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
                                                         -- STREAM_SELECT  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
-                                                            -- ASSIGN  |PARTITIONED|
-                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                       -- STREAM_PROJECT  |PARTITIONED|
                         -- STREAM_SELECT  |PARTITIONED|
@@ -54,9 +53,8 @@
                                               -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- STREAM_SELECT  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f8b50c6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -0,0 +1,133 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$55(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$88]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$88(ASC), $$49(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$88]  |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  |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  |PARTITIONED|
+                                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |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|
+                                              -- SPLIT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |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[$$88]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$88(ASC), $$49(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$88]  |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  |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  |PARTITIONED|
+                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- BTREE_SEARCH  |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|
+                                                        -- SPLIT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..35e1ec6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$60(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|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |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_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
index 008be34..45b3992 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/rtree-index-join/spatial-self-intersect-point-index-only.plan
@@ -26,12 +26,10 @@
                                                   -- STREAM_PROJECT  |PARTITIONED|
                                                     -- ASSIGN  |PARTITIONED|
                                                       -- STREAM_PROJECT  |PARTITIONED|
-                                                        -- ASSIGN  |PARTITIONED|
-                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
                                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- DATASOURCE_SCAN  |PARTITIONED|
-                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- STREAM_SELECT  |PARTITIONED|
@@ -49,9 +47,7 @@
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-ngram-index_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-ngram-index_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-ngram-index_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/skip-index/skip-ngram-index_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8361112
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/split-materialization_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$79(ASC), $$80(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$79(ASC), $$80(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|
+                                -- REPLICATE  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- STREAM_SELECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |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  |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|
+                                    -- 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|
+                                                      -- DATASOURCE_SCAN  |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  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4c9eff3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/exists_ps.plan
@@ -0,0 +1,157 @@
+-- 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[$$164]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$164]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$141]  |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[$$161]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$161]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$155]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- STREAM_SELECT  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$150][$$147]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
+                                                              -- ASSIGN  |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  |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  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |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[$$164]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$164]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$141]  |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[$$161]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- HASH_PARTITION_EXCHANGE [$$161]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$155]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$155(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$150][$$147]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$150]  |PARTITIONED|
+                                                                        -- ASSIGN  |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  |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  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f4ab7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_1_ps.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$18(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 [$$13][$$19]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |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|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..b4a8de2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_2_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |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|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- 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[$$19]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                              -- ASSIGN  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f4ab7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_3_ps.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$18(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 [$$13][$$19]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |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|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..01a0df8
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_4_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$22(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[$$18]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$19][$#1]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- 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[$$18]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$18]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$19][$#1]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                              -- ASSIGN  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f4ab7c2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_5_ps.plan
@@ -0,0 +1,41 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+      -- RANGE_PARTITION_EXCHANGE [$$18(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 [$$13][$$19]  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- DATASOURCE_SCAN  |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|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- HYBRID_HASH_JOIN [$$13][$$19]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- UNNEST  |UNPARTITIONED|
+                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..b4a8de2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_as_or_6_ps.plan
@@ -0,0 +1,67 @@
+-- DISTRIBUTE_RESULT  |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|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$19]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                      -- ASSIGN  |UNPARTITIONED|
+                                        -- UNNEST  |UNPARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
+          -- 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[$$19]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- STREAM_SELECT  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$19]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$20][$#1]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$20]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$#1]  |PARTITIONED|
+                                              -- ASSIGN  |UNPARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|


[21/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
[ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

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

details:
- new plan for sort operation which includes sampling and
replicating the stream of data to be sorted. Sort-merge connector
is removed from the plan. The sorted result now is in multiple partitions.
- new optimization rule to check whether full parallel sort is applicable.
- new Forward operator to read the replicated sort input stream and
to receive the ouput of the sampling.
- new sequential merge connector to merge a globally ordered result residing
in multiple partitions (in addition to the connector's partition computer).
- "asterix-lang-aql/pom.xml" is changed as a result of refactoring
code related to the range map handling.
- new private sampling function to generate the range map object
(local & global functions) & their type computers.

user model changes:
- new compiler property is added to enable and disable parallel sort.

interface changes:
- "ILogicalOperatorVisitor.java" includes Forward Operator.
- "ITuplePartitionComputer.java" includes initialize() to enable partitioner
to do some initialization. FieldRangePartitionComputerFactory uses it to
pick a range map.
- "ITuplePartitionComputerFactory.java". createPartitioner() is changed to
createPartitioner(IHyracksTaskContext hyracksTaskContext). Context is needed
for transferring the range map throught the context.

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


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

Branch: refs/heads/master
Commit: 80225e2c27d77514ecaa774235951187ef524193
Parents: adfb633
Author: Ali Alsuliman <al...@gmail.com>
Authored: Mon Oct 15 14:17:07 2018 -0700
Committer: Ali Alsuliman <al...@gmail.com>
Committed: Mon Oct 15 21:17:44 2018 -0700

----------------------------------------------------------------------
 .../asterix/optimizer/base/RuleCollections.java |    7 +-
 .../rules/CheckFullParallelSortRule.java        |   79 +
 .../rules/FixReplicateOperatorOutputsRule.java  |    2 +-
 .../SweepIllegalNonfunctionalFunctions.java     |   19 +-
 .../subplan/InlineAllNtsInSubplanVisitor.java   |    7 +
 ...neLeftNtsInSubplanJoinFlatteningVisitor.java |   35 +-
 .../SubplanSpecialFlatteningCheckVisitor.java   |   11 +-
 .../LangExpressionToPlanTranslator.java         |    6 +-
 .../apache/asterix/api/common/APIFramework.java |   30 +-
 .../app/resource/OperatorResourcesComputer.java |    1 +
 .../app/resource/PlanStagesGenerator.java       |   90 +-
 .../app/resource/RequiredCapacityVisitor.java   |    7 +
 .../app/translator/RequestParameters.java       |    2 +-
 .../asterix-app/src/main/resources/cc.conf      |    1 +
 .../dataflow/TestPartitionComputerFactory.java  |    3 +-
 .../asterix/test/optimizer/OptimizerTest.java   |    5 +
 ...-probe-pidx-with-join-btree-sidx_01_ps.sqlpp |   76 +
 ...-probe-pidx-with-join-btree-sidx_02_ps.sqlpp |   78 +
 .../btree-index/btree-secondary-68_ps.sqlpp     |   56 +
 ...ree-sidx-idxonly-01-disable-idxonly_ps.sqlpp |   68 +
 .../btree-index/btree-sidx-idxonly-01_ps.sqlpp  |   66 +
 .../queries/introhashpartitionmerge.sqlpp       |    6 +-
 .../ngram-contains-panic_ps.sqlpp               |   51 +
 .../ngram-contains_ps.sqlpp                     |   51 +
 .../olist-edit-distance-check-panic_ps.sqlpp    |   63 +
 .../olist-edit-distance-check_ps.sqlpp          |   63 +
 .../olist-edit-distance-panic_ps.sqlpp          |   63 +
 .../olist-edit-distance_ps.sqlpp                |   63 +
 .../olist-fuzzyeq-edit-distance_ps.sqlpp        |   65 +
 .../inverted-index-basic/word-contains_ps.sqlpp |   51 +
 ...olist-edit-distance-check-let-panic_ps.sqlpp |   65 +
 .../olist-edit-distance-check-let_ps.sqlpp      |   65 +
 ...ith-join-edit-distance-check-idx_01_ps.sqlpp |   81 +
 ...pidx-with-join-jaccard-check-idx_01_ps.sqlpp |   81 +
 .../inverted-index-join/ngram-contains_ps.sqlpp |   52 +
 .../optimizerts/queries/loj-core_ps.sqlpp       |   52 +
 .../optimizerts/queries/loj-sugar_ps.sqlpp      |   48 +
 ...-probe-pidx-with-join-btree-sidx_01_ps.sqlpp |   75 +
 ...-probe-pidx-with-join-btree-sidx_02_ps.sqlpp |   75 +
 .../ngram-contains-panic_ps.sqlpp               |   56 +
 .../ngram-contains_ps.sqlpp                     |   56 +
 .../olist-edit-distance-check-panic_ps.sqlpp    |   70 +
 .../olist-edit-distance-check_ps.sqlpp          |   70 +
 .../olist-edit-distance-panic_ps.sqlpp          |   70 +
 .../olist-edit-distance_ps.sqlpp                |   70 +
 .../olist-fuzzyeq-edit-distance_ps.sqlpp        |   70 +
 .../inverted-index-basic/word-contains_ps.sqlpp |   56 +
 ...olist-edit-distance-check-let-panic_ps.sqlpp |   72 +
 .../olist-edit-distance-check-let_ps.sqlpp      |   72 +
 ...ith-join-edit-distance-check-idx_01_ps.sqlpp |   76 +
 ...pidx-with-join-jaccard-check-idx_01_ps.sqlpp |   76 +
 ...-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp |   82 +
 ...-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp |   82 +
 ...robe-pidx-with-join-btree-sidx_01_1_ps.sqlpp |   74 +
 ...robe-pidx-with-join-btree-sidx_01_2_ps.sqlpp |   75 +
 ...robe-pidx-with-join-btree-sidx_02_1_ps.sqlpp |   74 +
 ...robe-pidx-with-join-btree-sidx_02_2_ps.sqlpp |   75 +
 .../ngram-contains-panic_ps.sqlpp               |   55 +
 .../ngram-contains_ps.sqlpp                     |   55 +
 .../inverted-index-basic/word-contains_ps.sqlpp |   55 +
 ...ith-join-edit-distance-check-idx_01_ps.sqlpp |   75 +
 .../ngram-contains_01_ps.sqlpp                  |   61 +
 .../ngram-contains_02_ps.sqlpp                  |   61 +
 .../ngram-contains_03_ps.sqlpp                  |   51 +
 .../ngram-contains_04_ps.sqlpp                  |   62 +
 ...-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp |   81 +
 ...-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp |   81 +
 ...robe-pidx-with-join-btree-sidx_01_1_ps.sqlpp |   69 +
 ...robe-pidx-with-join-btree-sidx_01_2_ps.sqlpp |   70 +
 ...robe-pidx-with-join-btree-sidx_02_1_ps.sqlpp |   69 +
 ...robe-pidx-with-join-btree-sidx_02_2_ps.sqlpp |   70 +
 .../ngram-contains-panic_ps.sqlpp               |   50 +
 .../ngram-contains_ps.sqlpp                     |   50 +
 .../inverted-index-basic/word-contains_ps.sqlpp |   50 +
 ...ith-join-edit-distance-check-idx_01_ps.sqlpp |   70 +
 .../ngram-contains_01_ps.sqlpp                  |   61 +
 .../ngram-contains_02_ps.sqlpp                  |   61 +
 .../ngram-contains_03_ps.sqlpp                  |   51 +
 .../ngram-contains_04_ps.sqlpp                  |   62 +
 ...-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp |   76 +
 ...-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp |   76 +
 .../btree-equi-join-non-enforced-05_ps.sqlpp    |   46 +
 .../btree-equi-join-non-enforced-06_ps.sqlpp    |   46 +
 .../btree-equi-join-non-enforced-07_ps.sqlpp    |   47 +
 .../btree-equi-join-non-enforced-08_ps.sqlpp    |   47 +
 .../btree-equi-join-non-enforced-09_ps.sqlpp    |   46 +
 .../btree-index-non-enforced-04_ps.sqlpp        |   44 +
 .../btree-index-non-enforced-05_ps.sqlpp        |   44 +
 .../btree-index-non-enforced-06_ps.sqlpp        |   44 +
 .../btree-index-non-enforced-07_ps.sqlpp        |   44 +
 .../btree-index-non-enforced-08_ps.sqlpp        |   44 +
 .../btree-index-non-enforced-09_ps.sqlpp        |   44 +
 .../btree-index-non-enforced-105_ps.sqlpp       |   46 +
 .../btree-index-non-enforced-10_ps.sqlpp        |   44 +
 .../btree-index-non-enforced-11_ps.sqlpp        |   44 +
 .../queries/orderby-desc-using-gby_ps.sqlpp     |   59 +
 ...rs-index-search-conjunctive-open_01_ps.sqlpp |   56 +
 ...rs-index-search-conjunctive-open_02_ps.sqlpp |   56 +
 .../orders-index-search-conjunctive_01_ps.sqlpp |   56 +
 .../orders-index-search-conjunctive_02_ps.sqlpp |   56 +
 .../parallel_sort_enabled_disabled.1.sqlpp      |   40 +
 .../parallel_sort_enabled_disabled.2.sqlpp      |   40 +
 .../q01_pricing_summary_report_nt_ps.sqlpp      |   80 +
 .../queries/query-ASTERIXDB-1806_ps.sqlpp       |   66 +
 .../queries/query-ASTERIXDB-2354_ps.sqlpp       |   29 +
 .../queries/query-ASTERIXDB-2408_ps.sqlpp       |   45 +
 .../queries/query-ASTERIXDB-810-2_ps.sqlpp      |   70 +
 .../queries/query-ASTERIXDB-810-3_ps.sqlpp      |   70 +
 .../queries/query-ASTERIXDB-810_ps.sqlpp        |   70 +
 .../optimizerts/queries/query-issue562_ps.sqlpp |  108 +
 ...-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp |   77 +
 ...-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp |   77 +
 .../skip-index/skip-ngram-index_ps.sqlpp        |   51 +
 .../queries/split-materialization_ps.sqlpp      |   61 +
 .../queries/subquery/exists_ps.sqlpp            |   72 +
 .../queries/subquery/in_as_or_1_ps.sqlpp        |   42 +
 .../queries/subquery/in_as_or_2_ps.sqlpp        |   43 +
 .../queries/subquery/in_as_or_3_ps.sqlpp        |   46 +
 .../queries/subquery/in_as_or_4_ps.sqlpp        |   47 +
 .../queries/subquery/in_as_or_5_ps.sqlpp        |   48 +
 .../queries/subquery/in_as_or_6_ps.sqlpp        |   49 +
 .../queries/subquery/in_correlated_ps.sqlpp     |   44 +
 .../optimizerts/queries/subquery/in_ps.sqlpp    |   44 +
 .../queries/subquery/not_exists_ps.sqlpp        |   73 +
 .../queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp |  131 +
 .../tpch/q12_shipping_broadcast_ps.sqlpp        |   53 +
 .../queries/tpch/q12_shipping_ps.sqlpp          |   53 +
 ...n-probe-pidx-with-join-btree-sidx_01_ps.plan |  109 +
 ...n-probe-pidx-with-join-btree-sidx_02_ps.plan |   77 +
 .../secondary-self-equi-join-index-only.plan    |   16 +-
 ...on-idxonly-to-sidx-idxonly-equi-join_01.plan |   50 +-
 .../btree-index/btree-secondary-68_ps.plan      |   69 +
 ...tree-sidx-idxonly-01-disable-idxonly_ps.plan |   47 +
 .../btree-index/btree-sidx-idxonly-01_ps.plan   |   65 +
 .../results/hints/fullparallelsort.plan         |  117 +
 .../results/hints/nofullparallel_hint.plan      |   38 +
 .../ngram-contains-panic_ps.plan                |   28 +
 .../inverted-index-basic/ngram-contains_ps.plan |   38 +
 .../olist-edit-distance-check-panic_ps.plan     |   28 +
 .../olist-edit-distance-check_ps.plan           |   38 +
 .../olist-edit-distance-panic_ps.plan           |   28 +
 .../olist-edit-distance_ps.plan                 |   38 +
 .../olist-fuzzyeq-edit-distance_ps.plan         |   38 +
 .../inverted-index-basic/word-contains_ps.plan  |   28 +
 .../olist-edit-distance-check-let-panic_ps.plan |   28 +
 .../olist-edit-distance-check-let_ps.plan       |   38 +
 ...with-join-edit-distance-check-idx_01_ps.plan |  165 +
 ...-pidx-with-join-jaccard-check-idx_01_ps.plan |  353 +
 .../inverted-index-join/ngram-contains_ps.plan  |   54 +
 .../optimizerts/results/loj-core_ps.plan        |   53 +
 .../optimizerts/results/loj-sugar_ps.plan       |   53 +
 ...n-probe-pidx-with-join-btree-sidx_01_ps.plan |   86 +
 ...n-probe-pidx-with-join-btree-sidx_02_ps.plan |   86 +
 .../ngram-contains-panic_ps.plan                |   32 +
 .../inverted-index-basic/ngram-contains_ps.plan |   42 +
 .../olist-edit-distance-check-panic_ps.plan     |   32 +
 .../olist-edit-distance-check_ps.plan           |   42 +
 .../olist-edit-distance-panic_ps.plan           |   32 +
 .../olist-edit-distance_ps.plan                 |   42 +
 .../olist-fuzzyeq-edit-distance_ps.plan         |   42 +
 .../inverted-index-basic/word-contains_ps.plan  |   32 +
 .../olist-edit-distance-check-let-panic_ps.plan |   32 +
 .../olist-edit-distance-check-let_ps.plan       |   42 +
 ...with-join-edit-distance-check-idx_01_ps.plan |  190 +
 ...-pidx-with-join-jaccard-check-idx_01_ps.plan |  358 +
 ...n-probe-pidx-with-join-rtree-sidx_01_ps.plan |   90 +
 ...n-probe-pidx-with-join-rtree-sidx_02_ps.plan |   92 +
 ...probe-pidx-with-join-btree-sidx_01_1_ps.plan |   86 +
 ...probe-pidx-with-join-btree-sidx_01_2_ps.plan |   86 +
 ...probe-pidx-with-join-btree-sidx_02_1_ps.plan |   86 +
 ...probe-pidx-with-join-btree-sidx_02_2_ps.plan |   86 +
 .../ngram-contains-panic_ps.plan                |   32 +
 .../inverted-index-basic/ngram-contains_ps.plan |   42 +
 .../inverted-index-basic/word-contains_ps.plan  |   32 +
 ...with-join-edit-distance-check-idx_01_ps.plan |  190 +
 .../ngram-contains_01_ps.plan                   |   50 +
 .../ngram-contains_02_ps.plan                   |   50 +
 .../ngram-contains_03_ps.plan                   |   54 +
 .../ngram-contains_04_ps.plan                   |   54 +
 ...n-probe-pidx-with-join-rtree-sidx_01_ps.plan |   90 +
 ...n-probe-pidx-with-join-rtree-sidx_02_ps.plan |   92 +
 ...probe-pidx-with-join-btree-sidx_01_1_ps.plan |   77 +
 ...probe-pidx-with-join-btree-sidx_01_2_ps.plan |   77 +
 ...probe-pidx-with-join-btree-sidx_02_1_ps.plan |   77 +
 ...probe-pidx-with-join-btree-sidx_02_2_ps.plan |   77 +
 .../ngram-contains-panic_ps.plan                |   28 +
 .../inverted-index-basic/ngram-contains_ps.plan |   38 +
 .../inverted-index-basic/word-contains_ps.plan  |   28 +
 ...with-join-edit-distance-check-idx_01_ps.plan |  165 +
 .../ngram-contains_01_ps.plan                   |   50 +
 .../ngram-contains_02_ps.plan                   |   54 +
 .../ngram-contains_03_ps.plan                   |   54 +
 .../ngram-contains_04_ps.plan                   |   54 +
 ...n-probe-pidx-with-join-rtree-sidx_01_ps.plan |   83 +
 ...n-probe-pidx-with-join-rtree-sidx_02_ps.plan |   83 +
 .../btree-equi-join-non-enforced-05_ps.plan     |   56 +
 .../btree-equi-join-non-enforced-06_ps.plan     |   56 +
 .../btree-equi-join-non-enforced-07_ps.plan     |   56 +
 .../btree-equi-join-non-enforced-08_ps.plan     |   58 +
 .../btree-equi-join-non-enforced-09_ps.plan     |   58 +
 .../btree-index-non-enforced-04_ps.plan         |   48 +
 .../btree-index-non-enforced-05_ps.plan         |   48 +
 .../btree-index-non-enforced-06_ps.plan         |   48 +
 .../btree-index-non-enforced-07_ps.plan         |   48 +
 .../btree-index-non-enforced-08_ps.plan         |   48 +
 .../btree-index-non-enforced-09_ps.plan         |   48 +
 .../btree-index-non-enforced-105_ps.plan        |   70 +
 .../btree-index-non-enforced-10_ps.plan         |   48 +
 .../btree-index-non-enforced-11_ps.plan         |   50 +
 .../results/orderby-desc-using-gby_ps.plan      |   44 +
 ...ers-index-search-conjunctive-open_01_ps.plan |   60 +
 ...ers-index-search-conjunctive-open_02_ps.plan |   60 +
 .../orders-index-search-conjunctive_01_ps.plan  |   60 +
 .../orders-index-search-conjunctive_02_ps.plan  |   60 +
 .../parallel_sort_enabled_disabled.1.plan       |   30 +
 .../parallel_sort_enabled_disabled.2.plan       |   12 +
 .../q01_pricing_summary_report_nt_ps.plan       |   74 +
 .../results/query-ASTERIXDB-1806_ps.plan        |   59 +
 .../results/query-ASTERIXDB-2354_ps.plan        |   98 +
 .../results/query-ASTERIXDB-2408_ps.plan        |   72 +
 .../results/query-ASTERIXDB-810-2_ps.plan       |   83 +
 .../results/query-ASTERIXDB-810-3_ps.plan       |   83 +
 .../results/query-ASTERIXDB-810_ps.plan         |   83 +
 .../optimizerts/results/query-issue562_ps.plan  |  135 +
 .../rtree-index-join/issue730-index-only.plan   |   22 +-
 ...n-probe-pidx-with-join-rtree-sidx_01_ps.plan |  133 +
 ...n-probe-pidx-with-join-rtree-sidx_02_ps.plan |   83 +
 ...spatial-self-intersect-point-index-only.plan |   16 +-
 .../results/skip-index/skip-ngram-index_ps.plan |   28 +
 .../results/split-materialization_ps.plan       |   67 +
 .../optimizerts/results/subquery/exists_ps.plan |  157 +
 .../results/subquery/in_as_or_1_ps.plan         |   41 +
 .../results/subquery/in_as_or_2_ps.plan         |   67 +
 .../results/subquery/in_as_or_3_ps.plan         |   41 +
 .../results/subquery/in_as_or_4_ps.plan         |   67 +
 .../results/subquery/in_as_or_5_ps.plan         |   41 +
 .../results/subquery/in_as_or_6_ps.plan         |   67 +
 .../results/subquery/in_correlated_ps.plan      |   84 +
 .../optimizerts/results/subquery/in_ps.plan     |   74 +
 .../results/subquery/not_exists_ps.plan         |  157 +
 .../results/tpcds/query-ASTERIXDB-1596_ps.plan  |   58 +
 .../results/tpch/q12_shipping_broadcast_ps.plan |   95 +
 .../results/tpch/q12_shipping_ps.plan           |   95 +
 .../big_object_insert.3.query.aql               |    4 +-
 .../prefix-search/prefix-search.3.query.aql     |    2 +-
 .../misc/stable_sort/stable_sort.3.query.aql    |    2 +-
 .../overlap_bins_gby_0.3.query.aql              |    4 +-
 .../p_sort_num_samples.1.ddl.sqlpp              |   47 +
 .../p_sort_num_samples.2.update.sqlpp           |   22 +
 .../p_sort_num_samples.3.query.sqlpp            |   26 +
 .../p_sort_seq_merge.1.ddl.sqlpp                |   36 +
 .../p_sort_seq_merge.2.update.sqlpp             |   42 +
 .../p_sort_seq_merge.3.query.sqlpp              |   24 +
 .../prefix-search/prefix-search.3.query.sqlpp   |    2 +-
 .../misc/stable_sort/stable_sort.3.query.sqlpp  |    3 +-
 .../object_remove/object_remove.3.query.sqlpp   |    2 +-
 .../single_dataset_with_index.13.query.sqlpp    |    2 +
 .../single_dataset_with_index.8.query.sqlpp     |    2 +
 .../overlap_bins_gby_0.3.query.sqlpp            |    2 +-
 .../cluster_state_1/cluster_state_1.1.regexadm  |    2 +
 .../cluster_state_1_full.1.regexadm             |    2 +
 .../cluster_state_1_less.1.regexadm             |    2 +
 .../p_sort_num_samples/p_sort_num_samples.3.adm | 6005 ++++++++++++++++
 .../p_sort_seq_merge/p_sort_seq_merge.3.adm     |    1 +
 .../misc/prefix-search/prefix-search.1.adm      |   90 +-
 .../results/misc/stable_sort/stable_sort.3.adm  | 6638 +++++++++---------
 .../misc/prefix-search/prefix-search.3.ast      |    5 +
 .../misc/stable_sort/stable_sort.3.ast          |   10 +
 .../overlap_bins_gby_0/overlap_bins_gby_0.3.ast |   10 +
 .../resources/runtimets/testsuite_sqlpp.xml     |   10 +
 .../common/config/CompilerProperties.java       |   21 +-
 .../asterix/common/exceptions/ErrorCode.java    |    1 +
 .../main/resources/asx_errormsg/en.properties   |    1 +
 .../ByteArrayAccessibleDataInputStream.java     |   33 -
 .../base/ByteArrayAccessibleInputStream.java    |   48 -
 asterixdb/asterix-lang-aql/pom.xml              |    8 -
 .../asterix/lang/aql/util/RangeMapBuilder.java  |  177 -
 .../asterix-lang-aql/src/main/javacc/AQL.jj     |    9 +-
 asterixdb/asterix-lang-common/pom.xml           |    4 +
 .../lang/common/clause/OrderbyClause.java       |   13 +-
 .../lang/common/util/RangeMapBuilder.java       |  175 +
 .../asterix-lang-sqlpp/src/main/javacc/SQLPP.jj |   14 +-
 .../asterix/om/functions/BuiltinFunctions.java  |   15 +
 .../impl/ListOfSamplesTypeComputer.java         |   45 +
 .../std/LocalSamplingAggregateDescriptor.java   |  173 +
 .../std/RangeMapAggregateDescriptor.java        |  317 +
 .../runtime/functions/FunctionCollection.java   |    4 +
 .../runtime/functions/FunctionTypeInferers.java |   26 +-
 .../core/algebra/base/LogicalOperatorTag.java   |    1 +
 .../core/algebra/base/OperatorAnnotations.java  |    3 +-
 .../core/algebra/base/PhysicalOperatorTag.java  |    2 +
 .../AbstractFunctionCallExpression.java         |   19 +-
 .../AggregateFunctionCallExpression.java        |    2 +
 .../logical/AbstractReplicateOperator.java      |   16 +-
 .../operators/logical/ForwardOperator.java      |  111 +
 .../visitors/CardinalityInferenceVisitor.java   |    8 +-
 .../visitors/FDsAndEquivClassesVisitor.java     |    7 +
 .../visitors/IsomorphismOperatorVisitor.java    |   13 +
 .../IsomorphismVariableMappingVisitor.java      |    7 +
 ...OperatorDeepCopyWithNewVariablesVisitor.java |    9 +
 .../visitors/LogicalPropertiesVisitor.java      |    9 +-
 .../visitors/OperatorDeepCopyVisitor.java       |    6 +
 .../visitors/PrimaryKeyVariablesVisitor.java    |    8 +-
 .../visitors/ProducedVariableVisitor.java       |    8 +-
 .../logical/visitors/SchemaVariableVisitor.java |    8 +
 .../visitors/SubstituteVariableVisitor.java     |    9 +
 .../logical/visitors/UsedVariableVisitor.java   |   54 +-
 .../physical/AbstractStableSortPOperator.java   |   54 +-
 .../operators/physical/ForwardPOperator.java    |  136 +
 .../RangePartitionExchangePOperator.java        |   55 +-
 .../RangePartitionMergeExchangePOperator.java   |   10 +-
 .../SequentialMergeExchangePOperator.java       |   76 +
 .../LogicalOperatorPrettyPrintVisitor.java      |    8 +
 .../LogicalOperatorPrettyPrintVisitorJson.java  |    9 +
 .../algebra/properties/LocalOrderProperty.java  |   15 +-
 .../core/algebra/properties/PropertiesUtil.java |   61 +-
 .../visitors/ILogicalOperatorVisitor.java       |    2 +
 .../core/config/AlgebricksConfig.java           |    2 +
 .../impl/ConnectorPolicyAssignmentPolicy.java   |    4 +-
 .../core/rewriter/base/HeuristicOptimizer.java  |   37 +-
 .../base/PhysicalOptimizationConfig.java        |   33 +-
 .../core/utils/DotFormatGenerator.java          |  152 +-
 .../core/utils/LogicalOperatorDotVisitor.java   |  390 +-
 .../rules/EnforceStructuralPropertiesRule.java  |  370 +-
 .../rules/ExtractCommonOperatorsRule.java       |   66 +
 .../rewriter/rules/InlineVariablesRule.java     |   30 +-
 .../SetAlgebricksPhysicalOperatorsRule.java     |    4 +
 ...placeNtsWithSubplanInputOperatorVisitor.java |    8 +-
 .../dataflow/value/ITuplePartitionComputer.java |   18 +
 .../value/ITuplePartitionComputerFactory.java   |    4 +-
 .../hyracks/api/exceptions/ErrorCode.java       |    3 +
 .../src/main/resources/errormsg/en.properties   |    3 +
 .../ByteArrayAccessibleDataInputStream.java     |   33 +
 .../util/ByteArrayAccessibleInputStream.java    |   48 +
 .../FieldHashPartitionComputerFactory.java      |    3 +-
 .../partition/OnePartitionComputerFactory.java  |   38 +
 .../RandomPartitionComputerFactory.java         |    3 +-
 .../partition/RepartitionComputerFactory.java   |    5 +-
 ...namicFieldRangePartitionComputerFactory.java |   48 +
 .../FieldRangePartitionComputerFactory.java     |   44 +-
 .../common/data/partition/range/IRangeMap.java  |   35 -
 .../common/data/partition/range/RangeMap.java   |  116 +-
 ...taticFieldRangePartitionComputerFactory.java |   38 +
 .../base/AbstractMToNConnectorDescriptor.java   |   18 +
 .../AbstractReplicateOperatorDescriptor.java    |    1 +
 .../DeterministicPartitionBatchManager.java     |   80 +
 .../collectors/SequentialMergeFrameReader.java  |   77 +
 ...wareMToNPartitioningConnectorDescriptor.java |    2 +-
 .../MToNBroadcastConnectorDescriptor.java       |   16 -
 .../MToNPartitioningConnectorDescriptor.java    |   19 +-
 ...NPartitioningMergingConnectorDescriptor.java |    2 +-
 ...titioningWithMessageConnectorDescriptor.java |    2 +-
 ...OneSequentialMergingConnectorDescriptor.java |   65 +
 .../std/connectors/PartitionDataWriter.java     |    1 +
 .../join/HybridHashJoinOperatorDescriptor.java  |   12 +-
 .../InMemoryHashJoinOperatorDescriptor.java     |    4 +-
 .../std/misc/ForwardOperatorDescriptor.java     |  246 +
 .../std/sort/AbstractExternalSortRunMerger.java |    9 +-
 358 files changed, 27634 insertions(+), 4310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/base/RuleCollections.java
----------------------------------------------------------------------
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 3c981d4..1010a84 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
@@ -24,6 +24,7 @@ import java.util.LinkedList;
 import java.util.List;
 
 import org.apache.asterix.common.dataflow.ICcApplicationContext;
+import org.apache.asterix.om.functions.BuiltinFunctions;
 import org.apache.asterix.optimizer.rules.AddEquivalenceClassForRecordConstructorRule;
 import org.apache.asterix.optimizer.rules.AsterixExtractFunctionsFromJoinConditionRule;
 import org.apache.asterix.optimizer.rules.AsterixInlineVariablesRule;
@@ -31,6 +32,7 @@ import org.apache.asterix.optimizer.rules.AsterixIntroduceGroupByCombinerRule;
 import org.apache.asterix.optimizer.rules.ByNameToByIndexFieldAccessRule;
 import org.apache.asterix.optimizer.rules.CancelUnnestWithNestedListifyRule;
 import org.apache.asterix.optimizer.rules.CheckFilterExpressionTypeRule;
+import org.apache.asterix.optimizer.rules.CheckFullParallelSortRule;
 import org.apache.asterix.optimizer.rules.CheckInsertUpsertReturningRule;
 import org.apache.asterix.optimizer.rules.ConstantFoldingRule;
 import org.apache.asterix.optimizer.rules.CountVarToCountOneRule;
@@ -338,7 +340,9 @@ public final class RuleCollections {
         physicalRewritesAllLevels.add(new SetAlgebricksPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new SetAsterixPhysicalOperatorsRule());
         physicalRewritesAllLevels.add(new AddEquivalenceClassForRecordConstructorRule());
-        physicalRewritesAllLevels.add(new EnforceStructuralPropertiesRule());
+        physicalRewritesAllLevels.add(new CheckFullParallelSortRule());
+        physicalRewritesAllLevels
+                .add(new EnforceStructuralPropertiesRule(BuiltinFunctions.RANGE_MAP, BuiltinFunctions.LOCAL_SAMPLING));
         physicalRewritesAllLevels.add(new RemoveSortInFeedIngestionRule());
         physicalRewritesAllLevels.add(new RemoveUnnecessarySortMergeExchange());
         physicalRewritesAllLevels.add(new PushProjectDownRule());
@@ -377,6 +381,7 @@ public final class RuleCollections {
         prepareForJobGenRewrites.add(new SetAlgebricksPhysicalOperatorsRule());
         prepareForJobGenRewrites
                 .add(new IsolateHyracksOperatorsRule(HeuristicOptimizer.hyraxOperatorsBelowWhichJobGenIsDisabled));
+        prepareForJobGenRewrites.add(new FixReplicateOperatorOutputsRule());
         prepareForJobGenRewrites.add(new ExtractCommonOperatorsRule());
         // Re-infer all types, so that, e.g., the effect of not-is-null is
         // propagated.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFullParallelSortRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFullParallelSortRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFullParallelSortRule.java
new file mode 100644
index 0000000..7b13a2b
--- /dev/null
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/CheckFullParallelSortRule.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.
+ */
+package org.apache.asterix.optimizer.rules;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.commons.lang3.mutable.Mutable;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+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.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
+
+/**
+ * <pre>
+ * Description:
+ *      This rule checks whether full parallel sort is applicable to {@link OrderOperator}. It disables full parallel
+ *      sort when a limit operator or running aggregate operator is present in the parents of the order operator.
+ * Pre-conditions:
+ *      None.
+ * Post-requirements:
+ *      1. {@link org.apache.hyracks.algebricks.rewriter.rules.EnforceStructuralPropertiesRule}
+ * </pre>
+ */
+public class CheckFullParallelSortRule implements IAlgebraicRewriteRule {
+    private final List<AbstractLogicalOperator> parents = new ArrayList<>();
+
+    @Override
+    public boolean rewritePre(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
+            throws AlgebricksException {
+        parents.add((AbstractLogicalOperator) opRef.getValue());
+        return false;
+    }
+
+    @Override
+    public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext ctx) throws AlgebricksException {
+        parents.remove(parents.size() - 1);
+        AbstractLogicalOperator orderOp = (AbstractLogicalOperator) opRef.getValue();
+        if (orderOp.getOperatorTag() == LogicalOperatorTag.ORDER
+                && !orderOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
+            // disable when sort output is consumed by limit & running agg op (result should be unpartitioned for limit)
+            AbstractLogicalOperator parent;
+            for (int i = parents.size() - 1; i >= 0; i--) {
+                parent = parents.get(i);
+                if (parent.getOperatorTag() == LogicalOperatorTag.LIMIT
+                        && ((LimitOperator) parent).isTopmostLimitOp()) {
+                    orderOp.getAnnotations().put(OperatorAnnotations.USE_DYNAMIC_RANGE, Boolean.FALSE);
+                    return true;
+                }
+                if (parent.getOperatorTag() == LogicalOperatorTag.RUNNINGAGGREGATE) {
+                    orderOp.getAnnotations().put(OperatorAnnotations.USE_DYNAMIC_RANGE, Boolean.FALSE);
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
index bd79a38..dc135c9 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/FixReplicateOperatorOutputsRule.java
@@ -140,7 +140,7 @@ public class FixReplicateOperatorOutputsRule implements IAlgebraicRewriteRule {
             boolean parentFixed = false;
             for (int oldParentIndex = 0; oldParentIndex < replicateOperator.getOutputs().size(); oldParentIndex++) {
                 if (parentsPathToReplicate.contains(replicateOperator.getOutputs().get(oldParentIndex))) {
-                    replicateOperator.getOutputs().get(oldParentIndex).setValue(replicateActualParent.getValue());
+                    replicateOperator.getOutputs().set(oldParentIndex, replicateActualParent);
                     parentFixed = true;
                     updateNumberOfParentsDone(replicateOperator);
                     break;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
index 8372851..0c91e9b 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/SweepIllegalNonfunctionalFunctions.java
@@ -34,11 +34,12 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCa
 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.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -101,12 +102,10 @@ public class SweepIllegalNonfunctionalFunctions extends AbstractExtractExprRule
     private class IllegalNonfunctionalFunctionSweeperOperatorVisitor implements ILogicalOperatorVisitor<Void, Void> {
 
         private void sweepExpression(ILogicalExpression expr, ILogicalOperator op) throws AlgebricksException {
-            if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL) {
-                if (!expr.isFunctional()) {
-                    AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
-                    throw new CompilationException(ErrorCode.COMPILATION_ERROR, fce.getSourceLocation(),
-                            "Found non-functional function " + fce.getFunctionIdentifier() + " in op " + op);
-                }
+            if (expr.getExpressionTag() == LogicalExpressionTag.FUNCTION_CALL && !expr.isFunctional()) {
+                AbstractFunctionCallExpression fce = (AbstractFunctionCallExpression) expr;
+                throw new CompilationException(ErrorCode.COMPILATION_ERROR, fce.getSourceLocation(),
+                        "Found non-functional function " + fce.getFunctionIdentifier() + " in op " + op);
             }
         }
 
@@ -308,6 +307,12 @@ public class SweepIllegalNonfunctionalFunctions extends AbstractExtractExprRule
         public Void visitTokenizeOperator(TokenizeOperator op, Void tag) throws AlgebricksException {
             return null;
         }
+
+        @Override
+        public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+            sweepExpression(op.getRangeMapExpression().getValue(), op);
+            return null;
+        }
     }
 
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
index 4d646f3..22ef303 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineAllNtsInSubplanVisitor.java
@@ -59,6 +59,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -642,6 +643,12 @@ class InlineAllNtsInSubplanVisitor implements IQueryOperatorVisitor<ILogicalOper
         return visitSingleInputOperator(op);
     }
 
+    @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                "Forward operator should have been disqualified for this rewriting!");
+    }
+
     /**
      * Wraps an AggregateOperator or RunningAggregateOperator with a group-by
      * operator where the group-by keys are variables in keyVarsToEnforce. Note

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
index 610fea2..b862a6f 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/InlineLeftNtsInSubplanJoinFlatteningVisitor.java
@@ -42,6 +42,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -52,6 +53,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.LimitOperato
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.MaterializeOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.NestedTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ProjectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.RunningAggregateOperator;
@@ -63,23 +65,19 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.TokenizeOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnionAllOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
 import org.apache.hyracks.algebricks.core.algebra.visitors.IQueryOperatorVisitor;
 
-/*
-    This visitor inlines the input <code>nts</code> in the query plan rooted
-    at the operator being visited, with the query plan rooted at the input
-    <code>subplanInputOperator</code>.
-
-    The visitor ensures that:
-    1. live variables at <code>subplanInputOperator</code> are
-    propagated to the top-most join operator in the query plan rooted
-    at the operator being visited;
-    2. no available tuple at <code>subplanInputOperator</code> get lost along the
-    pipeline to the top-most join operator in the query plan rooted
-    at the operator being visited.
-*/
+/**
+ * This visitor inlines the input {@code nts} in the query plan rooted at the operator being visited,
+ * with the query plan rooted at the input {@code subplanInputOperator}.
+ *
+ * The visitor ensures that:
+ * 1. live variables at {@code subplanInputOperator} are propagated to the top-most join operator in the query plan
+ * rooted at the operator being visited.
+ * 2. no available tuple at {@code subplanInputOperator} get lost along the pipeline to the top-most join operator
+ * in the query plan rooted at the operator being visited.
+ */
 class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisitor<ILogicalOperator, Void> {
     // The optimization context.
     private final IOptimizationContext context;
@@ -380,6 +378,12 @@ class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisit
         return visitSingleInputOperator(op);
     }
 
+    @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        throw new UnsupportedOperationException(
+                "Nested subplans with a forward operator should have been disqualified for this rewriting!");
+    }
+
     private ILogicalOperator visitSingleInputOperator(ILogicalOperator op) throws AlgebricksException {
         if (op.getInputs().size() == 1) {
             // Deals with single input operators.
@@ -395,8 +399,7 @@ class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisit
     }
 
     /**
-     * Inject varaibles to indicate non-matches for the right branch of
-     * a left-outer join.
+     * Inject variables to indicate non-matches for the right branch of a left-outer join.
      *
      * @param joinOp
      *            the leftouter join operator.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
index 44bfbe4..e2b104d 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/optimizer/rules/subplan/SubplanSpecialFlatteningCheckVisitor.java
@@ -18,16 +18,19 @@
  */
 package org.apache.asterix.optimizer.rules.subplan;
 
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 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.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -229,6 +232,12 @@ class SubplanSpecialFlatteningCheckVisitor implements IQueryOperatorVisitor<Bool
         return visitInputs(op);
     }
 
+    @Override
+    public Boolean visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        throw new CompilationException(ErrorCode.COMPILATION_ERROR, op.getSourceLocation(),
+                "Forward operator should have been disqualified for this rewriting!");
+    }
+
     private boolean visitInputs(ILogicalOperator op) throws AlgebricksException {
         for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
             if (childRef.getValue().accept(this, null)) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/LangExpressionToPlanTranslator.java
----------------------------------------------------------------------
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 91de474..19fe02d 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
@@ -38,7 +38,6 @@ 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.functions.FunctionSignature;
-import org.apache.asterix.lang.aql.util.RangeMapBuilder;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Expression.Kind;
 import org.apache.asterix.lang.common.base.ILangExpression;
@@ -69,6 +68,7 @@ import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.util.FunctionUtil;
+import org.apache.asterix.lang.common.util.RangeMapBuilder;
 import org.apache.asterix.lang.common.visitor.base.AbstractQueryExpressionVisitor;
 import org.apache.asterix.metadata.MetadataManager;
 import org.apache.asterix.metadata.declared.DataSource;
@@ -1163,9 +1163,9 @@ class LangExpressionToPlanTranslator
         }
         if (oc.getRangeMap() != null) {
             Iterator<OrderModifier> orderModifIter = oc.getModifierList().iterator();
-            boolean ascending = (orderModifIter.next() == OrderModifier.ASC);
+            boolean ascending = orderModifIter.next() == OrderModifier.ASC;
             RangeMapBuilder.verifyRangeOrder(oc.getRangeMap(), ascending);
-            ord.getAnnotations().put(OperatorAnnotations.USE_RANGE_CONNECTOR, oc.getRangeMap());
+            ord.getAnnotations().put(OperatorAnnotations.USE_STATIC_RANGE, oc.getRangeMap());
         }
         return new Pair<>(ord, null);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/main/java/org/apache/asterix/api/common/APIFramework.java
----------------------------------------------------------------------
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 cfec2de..9727d46 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
@@ -36,7 +36,6 @@ import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
 import org.apache.asterix.api.http.server.ResultUtil;
 import org.apache.asterix.common.api.INodeJobTracker;
 import org.apache.asterix.common.config.CompilerProperties;
-import org.apache.asterix.common.config.OptimizationConfUtil;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.exceptions.CompilationException;
@@ -136,7 +135,8 @@ public class APIFramework {
     private static final Set<String> CONFIGURABLE_PARAMETER_NAMES =
             ImmutableSet.of(CompilerProperties.COMPILER_JOINMEMORY_KEY, CompilerProperties.COMPILER_GROUPMEMORY_KEY,
                     CompilerProperties.COMPILER_SORTMEMORY_KEY, CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
-                    CompilerProperties.COMPILER_PARALLELISM_KEY, FunctionUtil.IMPORT_PRIVATE_FUNCTIONS,
+                    CompilerProperties.COMPILER_PARALLELISM_KEY, CompilerProperties.COMPILER_SORT_PARALLEL_KEY,
+                    CompilerProperties.COMPILER_SORT_SAMPLES_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,
@@ -336,12 +336,17 @@ public class APIFramework {
         int textSearchFrameLimit = getFrameLimit(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY,
                 (String) querySpecificConfig.get(CompilerProperties.COMPILER_TEXTSEARCHMEMORY_KEY),
                 compilerProperties.getTextSearchMemorySize(), frameSize, MIN_FRAME_LIMIT_FOR_TEXTSEARCH, sourceLoc);
-        final PhysicalOptimizationConfig physOptConf = OptimizationConfUtil.getPhysicalOptimizationConfig();
+        int sortNumSamples = getSortSamples(compilerProperties, querySpecificConfig);
+        boolean fullParallelSort = getSortParallel(compilerProperties, querySpecificConfig);
+
+        final PhysicalOptimizationConfig physOptConf = new PhysicalOptimizationConfig();
         physOptConf.setFrameSize(frameSize);
         physOptConf.setMaxFramesExternalSort(sortFrameLimit);
         physOptConf.setMaxFramesExternalGroupBy(groupFrameLimit);
         physOptConf.setMaxFramesForJoin(joinFrameLimit);
         physOptConf.setMaxFramesForTextSearch(textSearchFrameLimit);
+        physOptConf.setSortParallel(fullParallelSort);
+        physOptConf.setSortSamples(sortNumSamples);
 
         return physOptConf;
     }
@@ -495,6 +500,25 @@ public class APIFramework {
         return parameter == null ? parallelismInConfiguration : integerIPropertyInterpreter.parse(parameter);
     }
 
+    private boolean getSortParallel(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_PARALLEL_KEY);
+        if (valueInQuery != null) {
+            return OptionTypes.BOOLEAN.parse(valueInQuery);
+        }
+        return compilerProperties.getSortParallel();
+    }
+
+    private int getSortSamples(CompilerProperties compilerProperties, Map<String, Object> querySpecificConfig) {
+        String valueInQuery = (String) querySpecificConfig.get(CompilerProperties.COMPILER_SORT_SAMPLES_KEY);
+        if (valueInQuery != null) {
+            int parsedNumSamples = OptionTypes.INTEGER.parse(valueInQuery);
+            if (parsedNumSamples > 0) {
+                return parsedNumSamples;
+            }
+        }
+        return compilerProperties.getSortSamples();
+    }
+
     // Validates if the query contains unsupported query parameters.
     private static Map<String, Object> validateConfig(Map<String, Object> config, SourceLocation sourceLoc)
             throws AlgebricksException {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
index 1913683..c7db521 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/OperatorResourcesComputer.java
@@ -86,6 +86,7 @@ public class OperatorResourcesComputer {
             case INDEX_INSERT_DELETE_UPSERT:
             case INSERT_DELETE_UPSERT:
             case INTERSECT:
+            case FORWARD:
                 return getOperatorRequiredMemory(operator, frameSize);
             case LEFT_OUTER_UNNEST_MAP:
             case UNNEST_MAP:

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
index 8b32375..0023a7a 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/PlanStagesGenerator.java
@@ -38,6 +38,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -67,13 +68,22 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultO
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 import org.apache.hyracks.util.annotations.NotThreadSafe;
 
+/**
+ * Visits the operator first. Then, it visits all its inputs (pre-order traversal). When it visits an operator, it adds
+ * the operator to the current stage. If the operator is a multi-stage operator, it also adds the operator to a queue
+ * to re-visit the operator again to create the other stage.
+ */
 @NotThreadSafe
 public class PlanStagesGenerator implements ILogicalOperatorVisitor<Void, Void> {
 
-    private static final int JOIN_FIRST_INPUT = 1;
-    private static final int JOIN_SECOND_INPUT = 2;
+    private static final int JOIN_NON_BLOCKING_INPUT = 0;
+    private static final int JOIN_BLOCKING_INPUT = 1;
+    private static final int JOIN_NUM_INPUTS = 2;
+    private static final int FORWARD_NON_BLOCKING_INPUT = 0;
+    private static final int FORWARD_BLOCKING_INPUT = 1;
+    private static final int FORWARD_NUM_INPUTS = 2;
     private final Set<ILogicalOperator> visitedOperators = new HashSet<>();
-    private final LinkedList<ILogicalOperator> pendingBlockingOperators = new LinkedList<>();
+    private final LinkedList<ILogicalOperator> pendingMultiStageOperators = new LinkedList<>();
     private final List<PlanStage> stages = new ArrayList<>();
     private PlanStage currentStage;
     private int stageCounter;
@@ -163,7 +173,7 @@ public class PlanStagesGenerator implements ILogicalOperatorVisitor<Void, Void>
 
     @Override
     public Void visitReplicateOperator(ReplicateOperator op, Void arg) throws AlgebricksException {
-        // Makes sure that the downstream of a replicate operator is only visited once.
+        // make sure that the downstream of a replicate operator is visited only once.
         if (!visitedOperators.contains(op)) {
             visitedOperators.add(op);
             visit(op);
@@ -175,7 +185,7 @@ public class PlanStagesGenerator implements ILogicalOperatorVisitor<Void, Void>
 
     @Override
     public Void visitSplitOperator(SplitOperator op, Void arg) throws AlgebricksException {
-        // Makes sure that the downstream of a split operator is only visited once.
+        // make sure that the downstream of a split operator is visited only once.
         if (!visitedOperators.contains(op)) {
             visitedOperators.add(op);
             visit(op);
@@ -300,59 +310,82 @@ public class PlanStagesGenerator implements ILogicalOperatorVisitor<Void, Void>
         return null;
     }
 
+    @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        visit(op);
+        return null;
+    }
+
     public List<PlanStage> getStages() {
         return stages;
     }
 
     private void visit(ILogicalOperator op) throws AlgebricksException {
         addToStage(op);
-        if (!pendingBlockingOperators.isEmpty()) {
-            final ILogicalOperator firstPending = pendingBlockingOperators.pop();
-            visitBlocking(firstPending);
+        if (!pendingMultiStageOperators.isEmpty()) {
+            final ILogicalOperator firstPending = pendingMultiStageOperators.pop();
+            visitMultiStageOp(firstPending);
         }
     }
 
-    private void visitBlocking(ILogicalOperator blockingOp) throws AlgebricksException {
+    private void visitMultiStageOp(ILogicalOperator multiStageOp) throws AlgebricksException {
         final PlanStage blockingOpStage = new PlanStage(++stageCounter);
-        blockingOpStage.getOperators().add(blockingOp);
+        blockingOpStage.getOperators().add(multiStageOp);
         stages.add(blockingOpStage);
         currentStage = blockingOpStage;
-        switch (blockingOp.getOperatorTag()) {
+        switch (multiStageOp.getOperatorTag()) {
             case INNERJOIN:
             case LEFTOUTERJOIN:
-                // visit only the second input
-                ILogicalOperator joinSecondInput = getJoinOperatorInput(blockingOp, JOIN_SECOND_INPUT);
-                joinSecondInput.accept(this, null);
+                // visit only the blocking input creating a new stage
+                ILogicalOperator newStageOperator = getInputAt(multiStageOp, JOIN_BLOCKING_INPUT, JOIN_NUM_INPUTS);
+                newStageOperator.accept(this, null);
                 break;
             case GROUP:
             case ORDER:
-                visitInputs(blockingOp);
+                visitInputs(multiStageOp);
+                break;
+            case FORWARD:
+                // visit only the blocking input creating a new stage
+                ILogicalOperator newStageOp = getInputAt(multiStageOp, FORWARD_BLOCKING_INPUT, FORWARD_NUM_INPUTS);
+                newStageOp.accept(this, null);
                 break;
             default:
-                throw new IllegalStateException("Unrecognized blocking operator: " + blockingOp.getOperatorTag());
+                throw new IllegalStateException("Unrecognized blocking operator: " + multiStageOp.getOperatorTag());
         }
     }
 
+    /**
+     * Adds the op argument to the current stage. If the operator is a multi-stage, it adds the operator to the pending
+     * list and continues on the branch that is non-blocking (i.e., the branch continuing on the same current stage)
+     * @param op to be added to the current stage
+     * @throws AlgebricksException
+     */
     private void addToStage(ILogicalOperator op) throws AlgebricksException {
         currentStage.getOperators().add(op);
         switch (op.getOperatorTag()) {
             case INNERJOIN:
             case LEFTOUTERJOIN:
-                pendingBlockingOperators.add(op);
+                pendingMultiStageOperators.add(op);
                 // continue on the same stage
-                final ILogicalOperator joinFirstInput = getJoinOperatorInput(op, JOIN_FIRST_INPUT);
-                joinFirstInput.accept(this, null);
+                final ILogicalOperator joinNonBlockingInput = getInputAt(op, JOIN_NON_BLOCKING_INPUT, JOIN_NUM_INPUTS);
+                joinNonBlockingInput.accept(this, null);
                 break;
             case GROUP:
                 if (isBlockingGroupBy((GroupByOperator) op)) {
-                    pendingBlockingOperators.add(op);
+                    pendingMultiStageOperators.add(op);
                     return;
                 }
                 // continue on the same stage
                 visitInputs(op);
                 break;
             case ORDER:
-                pendingBlockingOperators.add(op);
+                pendingMultiStageOperators.add(op);
+                break;
+            case FORWARD:
+                pendingMultiStageOperators.add(op);
+                // continue on the same current stage through the branch that is non-blocking
+                ILogicalOperator nonBlockingInput = getInputAt(op, FORWARD_NON_BLOCKING_INPUT, FORWARD_NUM_INPUTS);
+                nonBlockingInput.accept(this, null);
                 break;
             default:
                 visitInputs(op);
@@ -397,15 +430,16 @@ public class PlanStagesGenerator implements ILogicalOperatorVisitor<Void, Void>
         return false;
     }
 
-    private ILogicalOperator getJoinOperatorInput(ILogicalOperator op, int inputNum) {
-        if (inputNum != JOIN_FIRST_INPUT && inputNum != JOIN_SECOND_INPUT) {
-            throw new IllegalArgumentException("invalid input number for join operator");
-        }
+    private ILogicalOperator getInputAt(ILogicalOperator op, int inputIndex, int numInputs) {
         final List<Mutable<ILogicalOperator>> inputs = op.getInputs();
-        if (inputs.size() != 2) {
-            throw new IllegalStateException("Join must have exactly two inputs. Current inputs: " + inputs.size());
+        int inSize = inputs.size();
+        if (inSize != numInputs) {
+            throw new IllegalStateException("Op must have exactly " + numInputs + " inputs. Current inputs: " + inSize);
+        }
+        if (inputIndex >= inSize) {
+            throw new IllegalArgumentException("invalid input index for operator");
         }
-        return op.getInputs().get(inputNum - 1).getValue();
+        return inputs.get(inputIndex).getValue();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
index 89fbcb2..368a244 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/resource/RequiredCapacityVisitor.java
@@ -36,6 +36,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -304,6 +305,12 @@ public class RequiredCapacityVisitor implements ILogicalOperatorVisitor<Void, Vo
         return null;
     }
 
+    @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        visitInternal(op, true);
+        return null;
+    }
+
     // Calculates the memory usage for exchange operators.
     private void calculateMemoryUsageForExchange(ExchangeOperator op) throws AlgebricksException {
         visitInternal(op, false);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
index 5f76568..d0adcda 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/RequestParameters.java
@@ -23,7 +23,6 @@ import java.io.DataOutputStream;
 import java.util.HashMap;
 import java.util.Map;
 
-import org.apache.asterix.external.library.java.base.ByteArrayAccessibleInputStream;
 import org.apache.asterix.external.parser.JSONDataParser;
 import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
 import org.apache.asterix.om.base.IAObject;
@@ -35,6 +34,7 @@ import org.apache.asterix.translator.ResultProperties;
 import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
 import org.apache.hyracks.api.result.IResultSet;
+import org.apache.hyracks.data.std.util.ByteArrayAccessibleInputStream;
 import org.apache.hyracks.data.std.util.ByteArrayAccessibleOutputStream;
 
 import com.fasterxml.jackson.databind.JsonNode;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/main/resources/cc.conf
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/resources/cc.conf b/asterixdb/asterix-app/src/main/resources/cc.conf
index 6971b2b..8877be8 100644
--- a/asterixdb/asterix-app/src/main/resources/cc.conf
+++ b/asterixdb/asterix-app/src/main/resources/cc.conf
@@ -53,6 +53,7 @@ compiler.sortmemory=320KB
 compiler.groupmemory=160KB
 compiler.joinmemory=256KB
 compiler.textsearchmemory=160KB
+compiler.sort.parallel=false
 messaging.frame.size=4096
 messaging.frame.count=512
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java
index 385f6a2..66b01a7 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/dataflow/TestPartitionComputerFactory.java
@@ -23,6 +23,7 @@ import java.util.Iterator;
 import java.util.List;
 
 import org.apache.hyracks.api.comm.IFrameTupleAccessor;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputer;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.exceptions.HyracksDataException;
@@ -41,7 +42,7 @@ public class TestPartitionComputerFactory implements ITuplePartitionComputerFact
     }
 
     @Override
-    public ITuplePartitionComputer createPartitioner() {
+    public ITuplePartitionComputer createPartitioner(IHyracksTaskContext ctx) {
         return new ITuplePartitionComputer() {
             private final List<Integer> destinations =
                     new ArrayList<Integer>(TestPartitionComputerFactory.this.destinations);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
index debb3f9..6e0413c 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/optimizer/OptimizerTest.java
@@ -60,6 +60,11 @@ import org.junit.runner.RunWith;
 import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
+/**
+ * Runs the optimization tests. The current configuration runs the tests with parallel sort disabled.
+ * Note: when adding a new test case and it includes sorting, provide another test case and enable parallel sort in the
+ * query by setting the property (append the test case name with "_ps")
+ */
 @RunWith(Parameterized.class)
 public class OptimizerTest {
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..4d8c4ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where (t1.countA /*+ indexnl */  = t2.countB)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..a5c5fe4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and
+ *                another for secondary btree index in index subtree. p_sort enabled.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68_ps.sqlpp
new file mode 100644
index 0000000..520a3f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-secondary-68_ps.sqlpp
@@ -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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index intersection with between operator (ASTERIXDB-2448). p_sort enabled.
+ *  Expected Result : Success
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.OrderType as
+ closed {
+  o_orderkey : bigint,
+  o_custkey : bigint,
+  o_orderstatus : string,
+  o_totalprice : double,
+  o_orderdate : string,
+  o_orderpriority : string,
+  o_clerk : string,
+  o_shippriority : bigint,
+  o_comment : string
+};
+
+create  dataset Orders(OrderType) primary key o_orderkey;
+
+create index idx_custkey on Orders (o_custkey) type btree;
+
+create index idx_orderpriority on Orders (o_orderpriority) type btree;
+
+set `compiler.sort.parallel` "true";
+
+select o_custkey, o_orderkey, o_orderstatus from Orders
+where
+  o_orderpriority = '1-URGENT' and
+  o_custkey between 40 and 43
+order by o_custkey, o_orderkey;
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.sqlpp
new file mode 100644
index 0000000..b986780
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.sqlpp
@@ -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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *                  : However, we set the "noindexonly" option to true. So, the index-only plan should not be triggered.
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord) primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+set noindexonly 'true';
+set `compiler.sort.parallel` "true";
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
new file mode 100644
index 0000000..e5c4e0c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/btree-index/btree-sidx-idxonly-01_ps.sqlpp
@@ -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.
+ */
+
+/*
+ *  Description     : Secondary BTree Index index-only selection plan verification test
+ *                  : The test is intended to verify that the secondary BTree index is used in the optimized query plan.
+ *                  : In this plan, we fetch PK and SK based on a select condition that utilizes a secondary index.
+ *                  : The plan should have two paths after the secondary index-lookup.
+ *                  : The left path:
+ *                      ... -> unnest-map (sidx) -> split -> unnest-map (pidx) -> select -> union -> ...
+ *                  : The right path:
+ *                      ... -> unnest-map (sidx) -> split ->                             -> union -> ...
+ *  Expected Result : Success
+ *
+*/
+
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+create type MyRecord as closed {
+  id: int64,
+  docid: int64,
+  val1: int64,
+  title: string,
+  point: point,
+  kwds: string,
+  line1: line,
+  line2: line,
+  poly1: polygon,
+  poly2: polygon,
+  rec: rectangle,
+  circle: circle
+};
+
+create dataset MyData(MyRecord) primary key id;
+
+create index btree_index_docid on MyData(docid) type btree;
+create index btree_index_val1 on MyData(val1) type btree;
+create index rtree_index_point on MyData(point) type rtree;
+create index rtree_index_rec on MyData(rec) type rtree;
+create index ngram_index_title on MyData(title) type ngram(3);
+create index keyword_index_title on MyData(title) type keyword;
+
+set `compiler.sort.parallel` "true";
+
+select element {"pk":o.id, "sk":o.docid}
+from MyData o
+where o.docid < 3
+order by o.id;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
index 103aae8..9921e58 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/introhashpartitionmerge.sqlpp
@@ -36,6 +36,9 @@ create  nodegroup group1 if not exists  on
 create  dataset TOKENSRANKEDADM(TOKENSRANKEDADMType) primary key rank on group1;
 
 write output to asterix_nc1:"rttest/introhashpartitionmerge.adm";
+
+SET `compiler.sort.parallel` "false";
+
 select element token2.rank
 from  TOKENSRANKEDADM as token1,
       (
@@ -44,5 +47,4 @@ from  TOKENSRANKEDADM as token1,
     order by tokenRanked.rank
 ) as token2
 where (token1.token = token2.token)
-order by token2.rank
-;
+order by token2.rank;


[06/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm
index 83f8e7e..a04e7dd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/stable_sort/stable_sort.3.adm
@@ -1,551 +1,557 @@
+{ "l_orderkey": 324, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-05-28", "l_receiptdate": "1992-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ross the slyly regular s" }
 { "l_orderkey": 1121, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-21", "l_commitdate": "1997-02-16", "l_receiptdate": "1997-04-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "odolites. slyly even accounts" }
+{ "l_orderkey": 1254, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51709.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-07", "l_commitdate": "1996-02-20", "l_receiptdate": "1996-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " platelets cajol" }
+{ "l_orderkey": 1411, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s against the" }
 { "l_orderkey": 1447, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 45108.2, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-01-05", "l_receiptdate": "1993-01-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "rts boost s" }
+{ "l_orderkey": 1827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 40707.4, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously ironic theodolites serve quickly af" }
+{ "l_orderkey": 2086, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lithely ironic acc" }
+{ "l_orderkey": 2304, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 46208.4, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quests are blithely alongside of" }
 { "l_orderkey": 2819, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 6.0, "l_extendedprice": 6601.2, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-22", "l_commitdate": "1994-08-02", "l_receiptdate": "1994-07-29", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "eas after the carefully express pack" }
+{ "l_orderkey": 3008, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 34106.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold packages. quic" }
+{ "l_orderkey": 3169, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18703.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-01-21", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular packages. ironi" }
 { "l_orderkey": 3264, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42907.8, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-07", "l_commitdate": "1996-12-12", "l_receiptdate": "1996-11-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "sleep carefully after the slyly final" }
-{ "l_orderkey": 4064, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-13", "l_receiptdate": "1997-01-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously f" }
-{ "l_orderkey": 1411, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s against the" }
-{ "l_orderkey": 2086, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.01, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-11-30", "l_commitdate": "1994-12-28", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lithely ironic acc" }
 { "l_orderkey": 3363, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 4.0, "l_extendedprice": 4400.8, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-30", "l_commitdate": "1995-11-17", "l_receiptdate": "1995-11-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " ironic dependencie" }
 { "l_orderkey": 3686, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 41807.6, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-04", "l_commitdate": "1998-08-11", "l_receiptdate": "1998-09-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "y silent foxes! carefully ruthless cour" }
 { "l_orderkey": 3878, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 6601.2, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-22", "l_receiptdate": "1997-07-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. regular instru" }
-{ "l_orderkey": 4931, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-18", "l_receiptdate": "1994-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s haggle al" }
-{ "l_orderkey": 5827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-09-24", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully special packages wake thin" }
-{ "l_orderkey": 5895, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gular deposits wake blithely carefully fin" }
-{ "l_orderkey": 1254, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 51709.4, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-07", "l_commitdate": "1996-02-20", "l_receiptdate": "1996-04-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " platelets cajol" }
-{ "l_orderkey": 3008, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 31.0, "l_extendedprice": 34106.2, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-14", "l_commitdate": "1995-12-11", "l_receiptdate": "1995-12-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold packages. quic" }
+{ "l_orderkey": 4064, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-17", "l_commitdate": "1996-12-13", "l_receiptdate": "1997-01-12", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "furiously f" }
 { "l_orderkey": 4098, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 50609.2, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-26", "l_commitdate": "1997-01-27", "l_receiptdate": "1997-02-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "e slyly blithely silent deposits. fluff" }
 { "l_orderkey": 4227, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2200.4, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-24", "l_commitdate": "1995-05-09", "l_receiptdate": "1995-05-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ep. specia" }
-{ "l_orderkey": 324, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-19", "l_commitdate": "1992-05-28", "l_receiptdate": "1992-05-12", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "ross the slyly regular s" }
-{ "l_orderkey": 1827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 40707.4, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-20", "l_commitdate": "1996-08-18", "l_receiptdate": "1996-08-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ously ironic theodolites serve quickly af" }
-{ "l_orderkey": 2304, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 42.0, "l_extendedprice": 46208.4, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-20", "l_commitdate": "1994-03-04", "l_receiptdate": "1994-02-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "quests are blithely alongside of" }
-{ "l_orderkey": 3169, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 18703.4, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-02", "l_commitdate": "1994-01-21", "l_receiptdate": "1994-03-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "usly regular packages. ironi" }
+{ "l_orderkey": 4931, "l_partkey": 200, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 50.0, "l_extendedprice": 55010.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-18", "l_receiptdate": "1994-12-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s haggle al" }
 { "l_orderkey": 5730, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 9901.8, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-02-02", "l_receiptdate": "1998-03-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s lose blithely. specia" }
 { "l_orderkey": 5764, "l_partkey": 200, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.1, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-12-24", "l_receiptdate": "1993-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ng to the fluffily qu" }
+{ "l_orderkey": 5827, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 26.0, "l_extendedprice": 28605.2, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-29", "l_commitdate": "1998-09-24", "l_receiptdate": "1998-07-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "arefully special packages wake thin" }
+{ "l_orderkey": 5895, "l_partkey": 200, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 20.0, "l_extendedprice": 22004.0, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-30", "l_commitdate": "1997-02-07", "l_receiptdate": "1997-05-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gular deposits wake blithely carefully fin" }
 { "l_orderkey": 5952, "l_partkey": 200, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53909.8, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-07-10", "l_receiptdate": "1997-07-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "e furiously regular" }
+{ "l_orderkey": 135, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 23082.99, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-01-12", "l_receiptdate": "1996-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " deposits believe. furiously regular p" }
 { "l_orderkey": 225, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49463.55, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-17", "l_commitdate": "1995-08-20", "l_receiptdate": "1995-08-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "the slyly even platelets use aro" }
+{ "l_orderkey": 231, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 54959.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic packages haggle fluffily a" }
 { "l_orderkey": 678, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 48.0, "l_extendedprice": 52761.12, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-28", "l_commitdate": "1993-04-04", "l_receiptdate": "1993-03-24", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ithely. slyly express foxes" }
 { "l_orderkey": 740, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31876.51, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-26", "l_commitdate": "1995-09-17", "l_receiptdate": "1995-10-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ntly bold pinto beans sleep quickl" }
+{ "l_orderkey": 804, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2198.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-04-30", "l_receiptdate": "1993-06-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "slyly silent " }
+{ "l_orderkey": 900, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 48364.36, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-03", "l_receiptdate": "1994-12-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " detect quick" }
+{ "l_orderkey": 964, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42868.41, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-06-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "se furiously regular instructions. blith" }
 { "l_orderkey": 999, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 45066.79, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-10-16", "l_commitdate": "1993-12-04", "l_receiptdate": "1993-11-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "us depths. carefully ironic instruc" }
 { "l_orderkey": 1024, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53860.31, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-01-26", "l_receiptdate": "1998-03-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ts. asymptotes nag fur" }
-{ "l_orderkey": 2535, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5495.95, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ", unusual reque" }
-{ "l_orderkey": 3495, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17587.04, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y bold dependencies; blithely idle sautern" }
-{ "l_orderkey": 4230, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 47265.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-14", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-03-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ses lose blithely slyly final e" }
-{ "l_orderkey": 4293, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 51661.93, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely pending deposits af" }
-{ "l_orderkey": 5379, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully final accounts haggle blithely. " }
-{ "l_orderkey": 135, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 21.0, "l_extendedprice": 23082.99, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-11", "l_commitdate": "1996-01-12", "l_receiptdate": "1996-02-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": " deposits believe. furiously regular p" }
 { "l_orderkey": 1348, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-14", "l_commitdate": "1998-07-10", "l_receiptdate": "1998-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "fter the regu" }
+{ "l_orderkey": 1728, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-26", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kly sly theodolites." }
+{ "l_orderkey": 1792, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 38471.65, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1994-01-20", "l_receiptdate": "1994-02-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e against the quic" }
 { "l_orderkey": 1954, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 12091.09, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-07", "l_commitdate": "1997-07-23", "l_receiptdate": "1997-08-25", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "y carefully ironi" }
 { "l_orderkey": 1985, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 30.0, "l_extendedprice": 32975.7, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-06", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-09-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "uickly. instr" }
-{ "l_orderkey": 3749, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-20", "l_receiptdate": "1995-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s. foxes sleep slyly unusual grouc" }
-{ "l_orderkey": 4002, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6595.14, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he slyly iro" }
-{ "l_orderkey": 4064, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3297.57, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its! quickly sp" }
-{ "l_orderkey": 804, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 2198.38, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-23", "l_commitdate": "1993-04-30", "l_receiptdate": "1993-06-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "slyly silent " }
-{ "l_orderkey": 900, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 48364.36, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-15", "l_commitdate": "1994-12-03", "l_receiptdate": "1994-12-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": " detect quick" }
 { "l_orderkey": 1988, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20884.61, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-03", "l_commitdate": "1995-12-10", "l_receiptdate": "1996-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "lly about the slyly thin instructions. f" }
+{ "l_orderkey": 2115, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14289.47, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-07", "l_commitdate": "1998-08-06", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans. even accounts abou" }
+{ "l_orderkey": 2535, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5495.95, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-09-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": ", unusual reque" }
 { "l_orderkey": 2752, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 41769.22, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-23", "l_commitdate": "1993-12-23", "l_receiptdate": "1994-03-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "es boost. slyly silent ideas" }
 { "l_orderkey": 2818, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 22.0, "l_extendedprice": 24182.18, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-28", "l_commitdate": "1995-03-10", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "egrate toward the carefully iron" }
+{ "l_orderkey": 2850, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4396.76, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al deposits cajole carefully quickly " }
 { "l_orderkey": 3333, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39570.84, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-20", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-12-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "foxes sleep neve" }
+{ "l_orderkey": 3495, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 16.0, "l_extendedprice": 17587.04, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-30", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-04-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "y bold dependencies; blithely idle sautern" }
+{ "l_orderkey": 3749, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "F", "l_shipdate": "1995-06-11", "l_commitdate": "1995-05-20", "l_receiptdate": "1995-06-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "s. foxes sleep slyly unusual grouc" }
+{ "l_orderkey": 4002, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6595.14, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-01", "l_commitdate": "1997-05-15", "l_receiptdate": "1997-07-31", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "he slyly iro" }
+{ "l_orderkey": 4064, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3297.57, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-04", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its! quickly sp" }
 { "l_orderkey": 4224, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29678.13, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-05", "l_commitdate": "1997-08-19", "l_receiptdate": "1997-09-30", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly special deposits sleep qui" }
+{ "l_orderkey": 4230, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 47265.17, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-03-14", "l_commitdate": "1992-05-13", "l_receiptdate": "1992-03-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ses lose blithely slyly final e" }
+{ "l_orderkey": 4293, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 47.0, "l_extendedprice": 51661.93, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-07", "l_commitdate": "1996-10-24", "l_receiptdate": "1996-09-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ithely pending deposits af" }
 { "l_orderkey": 5028, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16487.85, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-02", "l_commitdate": "1992-07-09", "l_receiptdate": "1992-08-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "gular, bold pinto bea" }
 { "l_orderkey": 5286, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1099.19, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-11-25", "l_commitdate": "1997-11-07", "l_receiptdate": "1997-12-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ly! furiously final pack" }
-{ "l_orderkey": 231, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 54959.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-11", "l_commitdate": "1994-12-14", "l_receiptdate": "1994-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "onic packages haggle fluffily a" }
-{ "l_orderkey": 964, "l_partkey": 199, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42868.41, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-06-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "se furiously regular instructions. blith" }
-{ "l_orderkey": 1728, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 34074.89, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-26", "l_commitdate": "1996-06-28", "l_receiptdate": "1996-08-14", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "kly sly theodolites." }
-{ "l_orderkey": 1792, "l_partkey": 199, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 35.0, "l_extendedprice": 38471.65, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-31", "l_commitdate": "1994-01-20", "l_receiptdate": "1994-02-17", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e against the quic" }
-{ "l_orderkey": 2115, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 5, "l_quantity": 13.0, "l_extendedprice": 14289.47, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-07", "l_commitdate": "1998-08-06", "l_receiptdate": "1998-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "into beans. even accounts abou" }
-{ "l_orderkey": 2850, "l_partkey": 199, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 4396.76, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-28", "l_commitdate": "1996-12-26", "l_receiptdate": "1996-11-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "al deposits cajole carefully quickly " }
+{ "l_orderkey": 5379, "l_partkey": 199, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43967.6, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-10-19", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "carefully final accounts haggle blithely. " }
 { "l_orderkey": 32, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 32.0, "l_extendedprice": 35142.08, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-14", "l_commitdate": "1995-10-07", "l_receiptdate": "1995-08-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lithely regular deposits. fluffily " }
+{ "l_orderkey": 295, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31847.51, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-09", "l_commitdate": "1994-12-08", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inst the carefully ironic pinto beans. blit" }
+{ "l_orderkey": 453, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 49418.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ifts wake carefully." }
+{ "l_orderkey": 608, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43927.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-04-11", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " alongside of the regular tithes. sly" }
 { "l_orderkey": 709, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-10", "l_commitdate": "1998-06-26", "l_receiptdate": "1998-08-09", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ily regular deposits. sauternes was accor" }
-{ "l_orderkey": 2533, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-06-02", "l_receiptdate": "1997-06-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ccounts. ironic, special accounts boo" }
-{ "l_orderkey": 3428, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-06-13", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly pending requests int" }
-{ "l_orderkey": 4551, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 29651.13, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y along the slyly even " }
+{ "l_orderkey": 738, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37338.46, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s against the ironic exc" }
+{ "l_orderkey": 805, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27454.75, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ide of the pending, sly requests. quickly f" }
 { "l_orderkey": 865, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-24", "l_commitdate": "1993-06-26", "l_receiptdate": "1993-08-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "y even accounts. quickly bold decoys" }
+{ "l_orderkey": 896, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10981.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-06-03", "l_receiptdate": "1993-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " quickly even theodolites. carefully regu" }
 { "l_orderkey": 1124, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 1098.19, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-10-06", "l_commitdate": "1998-10-02", "l_receiptdate": "1998-10-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " instructions cajole qu" }
+{ "l_orderkey": 1314, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-05-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "equests nag across the furious" }
+{ "l_orderkey": 1316, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 36240.27, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-31", "l_commitdate": "1994-01-23", "l_receiptdate": "1994-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "manently; blithely special deposits" }
 { "l_orderkey": 1345, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-27", "l_commitdate": "1993-01-23", "l_receiptdate": "1993-01-06", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "sly. furiously final accounts are blithely " }
+{ "l_orderkey": 1891, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " accounts are furiou" }
+{ "l_orderkey": 2277, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". quickly unusual deposi" }
 { "l_orderkey": 2407, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-24", "l_commitdate": "1998-08-18", "l_receiptdate": "1998-10-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "tructions wake stealt" }
+{ "l_orderkey": 2533, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-26", "l_commitdate": "1997-06-02", "l_receiptdate": "1997-06-24", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ccounts. ironic, special accounts boo" }
 { "l_orderkey": 3011, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-21", "l_commitdate": "1992-02-23", "l_receiptdate": "1992-05-15", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nusual sentiments. carefully bold idea" }
 { "l_orderkey": 3200, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 16.0, "l_extendedprice": 17571.04, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-28", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-03-11", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ly against the quiet packages. blith" }
-{ "l_orderkey": 4263, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "d accounts. daringly regular accounts hagg" }
-{ "l_orderkey": 453, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 49418.55, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-08-20", "l_receiptdate": "1997-07-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ifts wake carefully." }
-{ "l_orderkey": 805, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 27454.75, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-05", "l_commitdate": "1995-09-30", "l_receiptdate": "1995-08-06", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ide of the pending, sly requests. quickly f" }
-{ "l_orderkey": 1891, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-03-05", "l_receiptdate": "1995-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " accounts are furiou" }
-{ "l_orderkey": 2277, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-27", "l_commitdate": "1995-03-16", "l_receiptdate": "1995-04-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". quickly unusual deposi" }
-{ "l_orderkey": 4002, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21963.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly even ins" }
-{ "l_orderkey": 5061, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8785.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-10-31", "l_receiptdate": "1993-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "regular foxes. ir" }
-{ "l_orderkey": 5761, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-14", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly bold accounts wake above the" }
-{ "l_orderkey": 295, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 31847.51, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-09", "l_commitdate": "1994-12-08", "l_receiptdate": "1994-12-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "inst the carefully ironic pinto beans. blit" }
-{ "l_orderkey": 608, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43927.6, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-21", "l_commitdate": "1996-04-11", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": " alongside of the regular tithes. sly" }
-{ "l_orderkey": 738, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37338.46, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-09", "l_commitdate": "1993-04-15", "l_receiptdate": "1993-07-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "s against the ironic exc" }
-{ "l_orderkey": 896, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 10981.9, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-07", "l_commitdate": "1993-06-03", "l_receiptdate": "1993-07-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": " quickly even theodolites. carefully regu" }
-{ "l_orderkey": 1314, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5490.95, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-26", "l_commitdate": "1994-08-06", "l_receiptdate": "1994-05-31", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "equests nag across the furious" }
-{ "l_orderkey": 1316, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 33.0, "l_extendedprice": 36240.27, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-31", "l_commitdate": "1994-01-23", "l_receiptdate": "1994-04-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "manently; blithely special deposits" }
 { "l_orderkey": 3396, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 31.0, "l_extendedprice": 34043.89, "l_discount": 0.05, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-23", "l_receiptdate": "1994-06-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "l, express pinto beans. quic" }
+{ "l_orderkey": 3428, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 4.0, "l_extendedprice": 4392.76, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-09", "l_commitdate": "1996-06-13", "l_receiptdate": "1996-06-02", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "sly pending requests int" }
 { "l_orderkey": 3494, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 43927.6, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-30", "l_commitdate": "1993-07-02", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "uests cajole blithely" }
 { "l_orderkey": 3943, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 16472.85, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-13", "l_commitdate": "1996-12-17", "l_receiptdate": "1997-02-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " grow fluffily according to the " }
+{ "l_orderkey": 4002, "l_partkey": 198, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 20.0, "l_extendedprice": 21963.8, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-15", "l_commitdate": "1997-05-20", "l_receiptdate": "1997-07-11", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "lly even ins" }
+{ "l_orderkey": 4263, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 15374.66, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-06", "l_commitdate": "1998-04-17", "l_receiptdate": "1998-05-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "d accounts. daringly regular accounts hagg" }
 { "l_orderkey": 4294, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 32945.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-12", "l_commitdate": "1992-11-06", "l_receiptdate": "1992-09-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "olites. bold foxes affix ironic theodolite" }
+{ "l_orderkey": 4551, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 27.0, "l_extendedprice": 29651.13, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-28", "l_commitdate": "1996-03-22", "l_receiptdate": "1996-05-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "y along the slyly even " }
+{ "l_orderkey": 5061, "l_partkey": 198, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 8.0, "l_extendedprice": 8785.52, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-07", "l_commitdate": "1993-10-31", "l_receiptdate": "1993-10-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "regular foxes. ir" }
 { "l_orderkey": 5159, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 36.0, "l_extendedprice": 39534.84, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-24", "l_commitdate": "1996-11-07", "l_receiptdate": "1997-02-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "packages wake." }
 { "l_orderkey": 5186, "l_partkey": 198, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 44.0, "l_extendedprice": 48320.36, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-23", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-10-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "old, final accounts cajole sl" }
+{ "l_orderkey": 5761, "l_partkey": 198, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 49.0, "l_extendedprice": 53811.31, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-14", "l_commitdate": "1998-08-20", "l_receiptdate": "1998-07-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly bold accounts wake above the" }
+{ "l_orderkey": 70, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14263.47, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-03", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly special packag" }
 { "l_orderkey": 192, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-02-03", "l_receiptdate": "1998-03-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. dependencies nag furiously alongside" }
+{ "l_orderkey": 258, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "silent frets nod daringly busy, bold" }
 { "l_orderkey": 518, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 39.0, "l_extendedprice": 42790.41, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-26", "l_commitdate": "1998-03-17", "l_receiptdate": "1998-03-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " the bold, special deposits are carefully " }
-{ "l_orderkey": 1671, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50470.74, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". slyly bold instructions boost. furiousl" }
-{ "l_orderkey": 2789, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cording to the careful de" }
-{ "l_orderkey": 2820, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g multipliers. final c" }
-{ "l_orderkey": 3138, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "inal foxes affix slyly. fluffily regul" }
-{ "l_orderkey": 3616, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly ironic accounts unwind b" }
-{ "l_orderkey": 3748, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans run carefully quic" }
-{ "l_orderkey": 5056, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "c theodolites. ironic a" }
-{ "l_orderkey": 5536, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 38401.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "c, final theo" }
+{ "l_orderkey": 548, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-11-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests haggle quickly eve" }
+{ "l_orderkey": 706, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1995-12-02", "l_receiptdate": "1995-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ckey players. requests above the" }
+{ "l_orderkey": 961, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "warhorses slee" }
 { "l_orderkey": 1283, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 21.0, "l_extendedprice": 23040.99, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-12", "l_commitdate": "1996-10-02", "l_receiptdate": "1996-10-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "fully regular " }
 { "l_orderkey": 1378, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 37304.46, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-08", "l_commitdate": "1996-04-23", "l_receiptdate": "1996-07-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "le furiously slyly final accounts. careful" }
+{ "l_orderkey": 1671, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 46.0, "l_extendedprice": 50470.74, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-13", "l_commitdate": "1996-10-14", "l_receiptdate": "1996-09-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": ". slyly bold instructions boost. furiousl" }
+{ "l_orderkey": 1892, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15360.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously about the furiously" }
 { "l_orderkey": 2080, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42790.41, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-22", "l_commitdate": "1993-09-09", "l_receiptdate": "1993-08-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ic deposits haggle slyly carefully eve" }
-{ "l_orderkey": 2882, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31818.51, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-10", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. furiously ironic" }
-{ "l_orderkey": 5313, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17555.04, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ckages wake carefully aga" }
-{ "l_orderkey": 5509, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3291.57, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " quickly fin" }
-{ "l_orderkey": 70, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 14263.47, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-03", "l_commitdate": "1994-02-13", "l_receiptdate": "1994-03-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "lyly special packag" }
-{ "l_orderkey": 258, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-13", "l_commitdate": "1994-02-23", "l_receiptdate": "1994-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "silent frets nod daringly busy, bold" }
-{ "l_orderkey": 706, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-06", "l_commitdate": "1995-12-02", "l_receiptdate": "1995-12-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ckey players. requests above the" }
 { "l_orderkey": 2180, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 26332.56, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-03", "l_commitdate": "1996-10-24", "l_receiptdate": "1997-01-19", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "uriously f" }
+{ "l_orderkey": 2789, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 23.0, "l_extendedprice": 25235.37, "l_discount": 0.02, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-25", "l_commitdate": "1998-05-10", "l_receiptdate": "1998-04-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "cording to the careful de" }
 { "l_orderkey": 2790, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 26332.56, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-04", "l_commitdate": "1994-10-10", "l_receiptdate": "1994-12-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ments. slyly f" }
+{ "l_orderkey": 2820, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-08", "l_commitdate": "1994-07-30", "l_receiptdate": "1994-08-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "g multipliers. final c" }
+{ "l_orderkey": 2882, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 31818.51, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-10", "l_commitdate": "1995-11-01", "l_receiptdate": "1995-10-02", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "kages. furiously ironic" }
+{ "l_orderkey": 3138, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-24", "l_commitdate": "1994-05-07", "l_receiptdate": "1994-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "inal foxes affix slyly. fluffily regul" }
+{ "l_orderkey": 3296, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ronic ideas across the" }
 { "l_orderkey": 3552, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19749.42, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-11", "l_commitdate": "1997-07-14", "l_receiptdate": "1997-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s deposits against the blithely unusual pin" }
+{ "l_orderkey": 3587, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5485.95, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely regular decoys above the " }
+{ "l_orderkey": 3616, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-05", "l_commitdate": "1994-04-24", "l_receiptdate": "1994-05-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly ironic accounts unwind b" }
+{ "l_orderkey": 3748, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-23", "l_commitdate": "1998-05-17", "l_receiptdate": "1998-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "pinto beans run carefully quic" }
 { "l_orderkey": 3969, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 28526.94, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-08", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "fluffily; braids detect." }
+{ "l_orderkey": 4064, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es boost. careful" }
 { "l_orderkey": 4194, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 43.0, "l_extendedprice": 47179.17, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-06", "l_commitdate": "1994-12-09", "l_receiptdate": "1994-11-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "olites are after the exp" }
 { "l_orderkey": 4773, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 36.0, "l_extendedprice": 39498.84, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-08", "l_commitdate": "1996-03-03", "l_receiptdate": "1996-05-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": " dependencies. quickly" }
-{ "l_orderkey": 548, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 2.0, "l_extendedprice": 2194.38, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-11-26", "l_commitdate": "1994-11-06", "l_receiptdate": "1994-12-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ests haggle quickly eve" }
-{ "l_orderkey": 961, "l_partkey": 197, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 32915.7, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-07-20", "l_receiptdate": "1995-07-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "warhorses slee" }
-{ "l_orderkey": 1892, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 14.0, "l_extendedprice": 15360.66, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-08", "l_commitdate": "1994-06-12", "l_receiptdate": "1994-04-27", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "furiously about the furiously" }
-{ "l_orderkey": 3296, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 43887.6, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-28", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-01-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ronic ideas across the" }
-{ "l_orderkey": 3587, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5485.95, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-03", "l_commitdate": "1996-07-05", "l_receiptdate": "1996-09-11", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ithely regular decoys above the " }
-{ "l_orderkey": 4064, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 32.0, "l_extendedprice": 35110.08, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-14", "l_commitdate": "1997-01-01", "l_receiptdate": "1997-01-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "es boost. careful" }
+{ "l_orderkey": 5056, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 19.0, "l_extendedprice": 20846.61, "l_discount": 0.04, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-24", "l_commitdate": "1997-05-05", "l_receiptdate": "1997-04-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "c theodolites. ironic a" }
 { "l_orderkey": 5185, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 40596.03, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-08", "l_receiptdate": "1997-08-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "gainst the courts dazzle care" }
-{ "l_orderkey": 1731, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25212.37, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rays? bold, express pac" }
-{ "l_orderkey": 4000, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44943.79, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-03-14", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ve the even, fi" }
-{ "l_orderkey": 4038, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43847.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t. slyly silent pinto beans amo" }
-{ "l_orderkey": 4289, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20827.61, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-06", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e carefully regular ideas. sl" }
-{ "l_orderkey": 4583, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-11-21", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "to beans haggle sly" }
-{ "l_orderkey": 4803, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 42.0, "l_extendedprice": 46039.98, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-27", "l_commitdate": "1996-05-05", "l_receiptdate": "1996-05-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": " accounts affix quickly ar" }
+{ "l_orderkey": 5313, "l_partkey": 197, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 16.0, "l_extendedprice": 17555.04, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-04", "l_commitdate": "1997-08-02", "l_receiptdate": "1997-10-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ckages wake carefully aga" }
+{ "l_orderkey": 5509, "l_partkey": 197, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 3291.57, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-06-14", "l_commitdate": "1994-05-11", "l_receiptdate": "1994-06-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " quickly fin" }
+{ "l_orderkey": 5536, "l_partkey": 197, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 38401.65, "l_discount": 0.07, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-19", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-06-05", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "c, final theo" }
 { "l_orderkey": 71, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 34.0, "l_extendedprice": 37270.46, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-05", "l_commitdate": "1998-04-22", "l_receiptdate": "1998-03-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "s cajole. " }
-{ "l_orderkey": 1060, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously. furiously regular in" }
-{ "l_orderkey": 1539, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 23019.99, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-05-10", "l_receiptdate": "1995-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts haggle. busy" }
-{ "l_orderkey": 2214, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "t the blithely" }
-{ "l_orderkey": 2306, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-27", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly " }
-{ "l_orderkey": 3106, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6577.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "symptotes. slyly bold platelets cajol" }
-{ "l_orderkey": 4613, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 51520.93, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-05-26", "l_receiptdate": "1998-07-09", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "uriously special requests wak" }
-{ "l_orderkey": 5922, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 9.0, "l_extendedprice": 9865.71, "l_discount": 0.07, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-04", "l_commitdate": "1997-01-20", "l_receiptdate": "1996-12-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "haggle slyly even packages. packages" }
-{ "l_orderkey": 5959, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 13.0, "l_extendedprice": 14250.47, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-29", "l_commitdate": "1992-07-13", "l_receiptdate": "1992-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ar forges. deposits det" }
-{ "l_orderkey": 5986, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 27404.75, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-16", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-06-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " instructions. slyly regular de" }
 { "l_orderkey": 259, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 3288.57, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-04", "l_commitdate": "1993-11-07", "l_receiptdate": "1993-10-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "ng slyly at the accounts." }
 { "l_orderkey": 482, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-19", "l_commitdate": "1996-05-05", "l_receiptdate": "1996-04-21", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "tructions near the final, regular ideas de" }
 { "l_orderkey": 549, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 18.0, "l_extendedprice": 19731.42, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-10-19", "l_commitdate": "1992-08-12", "l_receiptdate": "1992-11-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "furiously according to the ironic, regular " }
 { "l_orderkey": 614, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 30.0, "l_extendedprice": 32885.7, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-16", "l_commitdate": "1993-02-08", "l_receiptdate": "1993-02-12", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "tructions are f" }
+{ "l_orderkey": 768, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 42751.41, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-27", "l_receiptdate": "1996-10-20", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "out the ironic" }
+{ "l_orderkey": 1060, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 8.0, "l_extendedprice": 8769.52, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-05-21", "l_commitdate": "1993-05-06", "l_receiptdate": "1993-06-10", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "iously. furiously regular in" }
+{ "l_orderkey": 1154, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-04", "l_commitdate": "1992-04-01", "l_receiptdate": "1992-04-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " even, special " }
+{ "l_orderkey": 1155, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 42751.41, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-01-29", "l_commitdate": "1998-01-03", "l_receiptdate": "1998-02-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ckly final pinto beans was." }
+{ "l_orderkey": 1539, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 23019.99, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-04-19", "l_commitdate": "1995-05-10", "l_receiptdate": "1995-04-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ounts haggle. busy" }
+{ "l_orderkey": 1731, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 23.0, "l_extendedprice": 25212.37, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-22", "l_commitdate": "1996-02-25", "l_receiptdate": "1996-05-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "rays? bold, express pac" }
 { "l_orderkey": 1828, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 11.0, "l_extendedprice": 12058.09, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-21", "l_commitdate": "1994-05-28", "l_receiptdate": "1994-08-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": " wake blithely " }
 { "l_orderkey": 2115, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.06, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-01", "l_commitdate": "1998-07-29", "l_receiptdate": "1998-09-04", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "de of the carefully bold accounts " }
+{ "l_orderkey": 2214, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 22.0, "l_extendedprice": 24116.18, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-30", "l_commitdate": "1998-07-02", "l_receiptdate": "1998-06-09", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "t the blithely" }
+{ "l_orderkey": 2306, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 54809.5, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-27", "l_commitdate": "1995-09-26", "l_receiptdate": "1995-08-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "y quickly " }
+{ "l_orderkey": 3106, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 6.0, "l_extendedprice": 6577.14, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1997-04-11", "l_receiptdate": "1997-02-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "symptotes. slyly bold platelets cajol" }
 { "l_orderkey": 3717, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 49328.55, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-19", "l_commitdate": "1998-07-22", "l_receiptdate": "1998-09-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "s the blithely unu" }
 { "l_orderkey": 3845, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 27.0, "l_extendedprice": 29597.13, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-20", "l_commitdate": "1992-07-17", "l_receiptdate": "1992-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "kages. care" }
+{ "l_orderkey": 4000, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 41.0, "l_extendedprice": 44943.79, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-02", "l_commitdate": "1992-03-14", "l_receiptdate": "1992-03-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ve the even, fi" }
 { "l_orderkey": 4034, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 7.0, "l_extendedprice": 7673.33, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-04", "l_commitdate": "1994-01-22", "l_receiptdate": "1994-04-01", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "y even theodolites. slyly regular instru" }
+{ "l_orderkey": 4038, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 40.0, "l_extendedprice": 43847.6, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-15", "l_commitdate": "1996-03-13", "l_receiptdate": "1996-01-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t. slyly silent pinto beans amo" }
 { "l_orderkey": 4128, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 5480.95, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-18", "l_commitdate": "1995-11-28", "l_receiptdate": "1995-10-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ake permanently " }
 { "l_orderkey": 4230, "l_partkey": 196, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 10.0, "l_extendedprice": 10961.9, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-11", "l_commitdate": "1992-04-11", "l_receiptdate": "1992-07-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ar packages are " }
+{ "l_orderkey": 4263, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-06-24", "l_commitdate": "1998-06-08", "l_receiptdate": "1998-07-14", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ideas for the carefully re" }
+{ "l_orderkey": 4289, "l_partkey": 196, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 20827.61, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-31", "l_commitdate": "1993-11-06", "l_receiptdate": "1994-01-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e carefully regular ideas. sl" }
 { "l_orderkey": 4390, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-07", "l_commitdate": "1995-06-22", "l_receiptdate": "1995-10-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ld braids haggle atop the for" }
+{ "l_orderkey": 4583, "l_partkey": 196, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 30693.32, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-29", "l_commitdate": "1994-11-21", "l_receiptdate": "1994-11-28", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "to beans haggle sly" }
+{ "l_orderkey": 4613, "l_partkey": 196, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 51520.93, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-03", "l_commitdate": "1998-05-26", "l_receiptdate": "19

<TRUNCATED>

[11/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-07_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- 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  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-08_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- 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  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-09_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- 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  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..dec1cb4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-105_ps.plan
@@ -0,0 +1,70 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$22(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- INTERSECT  |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  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |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  |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  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- INTERSECT  |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  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |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  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-10_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- 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  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4adc026
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-11_ps.plan
@@ -0,0 +1,50 @@
+-- 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|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |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  |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  |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  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..93e872a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orderby-desc-using-gby_ps.plan
@@ -0,0 +1,44 @@
+-- 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[$$39, $$40]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$39, $$40]  |PARTITIONED|
+                          -- STREAM_PROJECT  |PARTITIONED|
+                            -- ASSIGN  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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[$$39, $$40]  |PARTITIONED|
+                                      {
+                                        -- AGGREGATE  |LOCAL|
+                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                      }
+                                -- HASH_PARTITION_EXCHANGE [$$39, $$40]  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2c7fcb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_01_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |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  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- 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|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |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  |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|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..386c64a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive-open_02_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |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  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- 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_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |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  |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  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2c7fcb7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_01_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |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  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- 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|
+                      -- STREAM_PROJECT  |PARTITIONED|
+                        -- STREAM_SELECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |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  |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|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..386c64a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/orders-index-search-conjunctive_02_ps.plan
@@ -0,0 +1,60 @@
+-- SINK  |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  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+-- 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_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- BTREE_SEARCH  |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  |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  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.plan
new file mode 100644
index 0000000..d5b2526
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.1.plan
@@ -0,0 +1,30 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$12(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|
+                          -- DATASOURCE_SCAN  |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|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.plan
new file mode 100644
index 0000000..32499ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/parallel_sort_enabled_disabled/parallel_sort_enabled_disabled.2.plan
@@ -0,0 +1,12 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$12(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$12(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- ASSIGN  |PARTITIONED|
+              -- STREAM_PROJECT  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- DATASOURCE_SCAN  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..390afe3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/q01_pricing_summary_report_nt_ps.plan
@@ -0,0 +1,74 @@
+-- SINK  |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), $$3(ASC)]  |PARTITIONED|
+              -- HASH_PARTITION_EXCHANGE [$$2, $$3]  |PARTITIONED|
+                -- ASSIGN  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- DATASOURCE_SCAN  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- 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[$$190, $$191]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$190, $$191]  |PARTITIONED|
+                          -- EXTERNAL_GROUP_BY[$$159, $$160]  |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  |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[$$190, $$191]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$190, $$191]  |PARTITIONED|
+                                    -- EXTERNAL_GROUP_BY[$$159, $$160]  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..5585ddd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-1806_ps.plan
@@ -0,0 +1,59 @@
+-- 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[$$149, $$150]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$149, $$150]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$117, $$118]  |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  |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[$$149, $$150]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$149, $$150]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$117, $$118]  |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  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..a110303
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2354_ps.plan
@@ -0,0 +1,98 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$70(ASC), $$71(ASC), $$72(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$70(ASC), $$71(ASC), $$72(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- UNION_ALL  |PARTITIONED|
+                      -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |PARTITIONED|
+                            -- STREAM_PROJECT  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- REPLICATE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ASSIGN  |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  |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  |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|
+                              -- UNION_ALL  |PARTITIONED|
+                                -- RANDOM_PARTITION_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- REPLICATE  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |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  |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  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..bd0ff12
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-2408_ps.plan
@@ -0,0 +1,72 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$39(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$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|
+                            -- PRE_CLUSTERED_GROUP_BY[$$37]  |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 [$$37][$$40]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |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[$$37]  |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 [$$37][$$40]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$40]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..ed8309c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-2_ps.plan
@@ -0,0 +1,83 @@
+-- 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[$$104, $$105]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$104(ASC), $$105(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$104, $$105]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$90, $$91]  |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 [$$90(ASC), $$91(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |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[$$104, $$105]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$104(ASC), $$105(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$104, $$105]  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$90, $$91]  |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 [$$90(ASC), $$91(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7b7285b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-ASTERIXDB-810-3_ps.plan
@@ -0,0 +1,83 @@
+-- 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[$$112, $$113]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$112(ASC), $$113(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$112, $$113]  |PARTITIONED|
+                              -- PRE_CLUSTERED_GROUP_BY[$$99, $$100]  |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 [$$99(ASC), $$100(ASC)]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |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[$$112, $$113]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$112(ASC), $$113(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$112, $$113]  |PARTITIONED|
+                                        -- PRE_CLUSTERED_GROUP_BY[$$99, $$100]  |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 [$$99(ASC), $$100(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[05/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast
index 1dc730f..edea622 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/prefix-search/prefix-search.3.ast
@@ -51,4 +51,9 @@ Orderby
     Field=l_linenumber
   ]
   ASC
+  FieldAccessor [
+    Variable [ Name=$l ]
+    Field=l_orderkey
+  ]
+  ASC
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast
index 87320ab..0460dc9 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/misc/stable_sort/stable_sort.3.ast
@@ -14,4 +14,14 @@ Orderby
     Field=l_partkey
   ]
   DESC
+  FieldAccessor [
+    Variable [ Name=$i ]
+    Field=l_orderkey
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$i ]
+    Field=l_linenumber
+  ]
+  ASC
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast
index 36fb0a1..9ea33d5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.ast
@@ -137,4 +137,14 @@ Orderby
     ]
   ]
   ASC
+  Variable [ Name=$itv ]
+  ASC
+  FunctionCall test.get-overlapping-interval@2[
+    FieldAccessor [
+      Variable [ Name=$gen0 ]
+      Field=bin
+    ]
+    Variable [ Name=$itv ]
+  ]
+  ASC
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
index 3236e62..f235d54 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/testsuite_sqlpp.xml
@@ -4339,6 +4339,16 @@
         <output-dir compare="Text">uuid</output-dir>
       </compilation-unit>
     </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_seq_merge">
+        <output-dir compare="Text">p_sort_seq_merge</output-dir>
+      </compilation-unit>
+    </test-case>
+    <test-case FilePath="misc">
+      <compilation-unit name="p_sort_num_samples">
+        <output-dir compare="Text">p_sort_num_samples</output-dir>
+      </compilation-unit>
+    </test-case>
   </test-group>
   <test-group name="index">
     <test-group name="index/validations">

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/config/CompilerProperties.java
----------------------------------------------------------------------
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 6d89ff5..4c58ad7 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
@@ -22,6 +22,7 @@ import static org.apache.hyracks.control.common.config.OptionTypes.*;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
 import org.apache.hyracks.api.config.IOption;
 import org.apache.hyracks.api.config.IOptionType;
 import org.apache.hyracks.api.config.Section;
@@ -58,7 +59,12 @@ public class CompilerProperties extends AbstractProperties {
                         + "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"),
-        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)");
+        COMPILER_STRINGOFFSET(INTEGER, 0, "Position of a first character in a String/Binary (0 or 1)"),
+        COMPILER_SORT_PARALLEL(BOOLEAN, AlgebricksConfig.SORT_PARALLEL, "Enabling/Disabling full parallel sort"),
+        COMPILER_SORT_SAMPLES(
+                INTEGER,
+                AlgebricksConfig.SORT_SAMPLES,
+                "The number of samples parallel sorting should " + "take from each partition");
 
         private final IOptionType type;
         private final Object defaultValue;
@@ -106,6 +112,10 @@ public class CompilerProperties extends AbstractProperties {
 
     public static final String COMPILER_PARALLELISM_KEY = Option.COMPILER_PARALLELISM.ini();
 
+    public static final String COMPILER_SORT_PARALLEL_KEY = Option.COMPILER_SORT_PARALLEL.ini();
+
+    public static final String COMPILER_SORT_SAMPLES_KEY = Option.COMPILER_SORT_SAMPLES.ini();
+
     public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
 
     public CompilerProperties(PropertiesAccessor accessor) {
@@ -140,4 +150,13 @@ public class CompilerProperties extends AbstractProperties {
         int value = accessor.getInt(Option.COMPILER_STRINGOFFSET);
         return value > 0 ? 1 : 0;
     }
+
+    public boolean getSortParallel() {
+        return accessor.getBoolean(Option.COMPILER_SORT_PARALLEL);
+    }
+
+    public int getSortSamples() {
+        int numSamples = accessor.getInt(Option.COMPILER_SORT_SAMPLES);
+        return numSamples > 0 ? numSamples : AlgebricksConfig.SORT_SAMPLES;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-common/src/main/java/org/apache/asterix/common/exceptions/ErrorCode.java
----------------------------------------------------------------------
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 dda6f7b..1bf2447 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
@@ -179,6 +179,7 @@ public class ErrorCode {
     public static final int COMPILATION_TYPE_MISMATCH_GENERIC = 1091;
     public static final int ILLEGAL_SET_PARAMETER = 1092;
     public static final int COMPILATION_TRANSLATION_ERROR = 1093;
+    public static final int RANGE_MAP_ERROR = 1094;
 
     // Feed errors
     public static final int DATAFLOW_ILLEGAL_STATE = 3001;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
----------------------------------------------------------------------
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 4c8c866..5629b97 100644
--- a/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
+++ b/asterixdb/asterix-common/src/main/resources/asx_errormsg/en.properties
@@ -166,6 +166,7 @@
 1090 = Field %1$s must be of an array of type %2$s but found to contain an item of type %3$s
 1092 = Parameter %1$s cannot be set
 1093 = A parser error has occurred. The detail exception: %1$s
+1094 = Cannot parse range map: %1$s
 
 # Feed Errors
 3001 = Illegal state.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
deleted file mode 100644
index b92aa6c..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleDataInputStream.java
+++ /dev/null
@@ -1,33 +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.external.library.java.base;
-
-import java.io.DataInputStream;
-
-public class ByteArrayAccessibleDataInputStream extends DataInputStream {
-
-    public ByteArrayAccessibleDataInputStream(ByteArrayAccessibleInputStream in) {
-        super(in);
-    }
-
-    public ByteArrayAccessibleInputStream getInputStream() {
-        return (ByteArrayAccessibleInputStream) in;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleInputStream.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleInputStream.java b/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleInputStream.java
deleted file mode 100644
index 62e354b..0000000
--- a/asterixdb/asterix-external-data/src/main/java/org/apache/asterix/external/library/java/base/ByteArrayAccessibleInputStream.java
+++ /dev/null
@@ -1,48 +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.external.library.java.base;
-
-import java.io.ByteArrayInputStream;
-
-public class ByteArrayAccessibleInputStream extends ByteArrayInputStream {
-
-    public ByteArrayAccessibleInputStream(byte[] buf, int offset, int length) {
-        super(buf, offset, length);
-    }
-
-    public void setContent(byte[] buf, int offset, int length) {
-        this.buf = buf;
-        this.pos = offset;
-        this.count = Math.min(offset + length, buf.length);
-        this.mark = offset;
-    }
-
-    public byte[] getArray() {
-        return buf;
-    }
-
-    public int getPosition() {
-        return pos;
-    }
-
-    public int getCount() {
-        return count;
-    }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-lang-aql/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/pom.xml b/asterixdb/asterix-lang-aql/pom.xml
index 6181fe6..f202192 100644
--- a/asterixdb/asterix-lang-aql/pom.xml
+++ b/asterixdb/asterix-lang-aql/pom.xml
@@ -158,10 +158,6 @@
       <artifactId>commons-io</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-dataflow-common</artifactId>
-    </dependency>
-    <dependency>
       <groupId>org.apache.asterix</groupId>
       <artifactId>asterix-om</artifactId>
       <version>${project.version}</version>
@@ -184,10 +180,6 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
-    <dependency>
-      <groupId>org.apache.hyracks</groupId>
-      <artifactId>hyracks-data-std</artifactId>
-    </dependency>
   </dependencies>
 
 </project>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java b/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
deleted file mode 100644
index 23b0066..0000000
--- a/asterixdb/asterix-lang-aql/src/main/java/org/apache/asterix/lang/aql/util/RangeMapBuilder.java
+++ /dev/null
@@ -1,177 +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.lang.aql.util;
-
-import java.io.DataOutput;
-import java.util.List;
-
-import org.apache.asterix.common.exceptions.CompilationException;
-import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
-import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
-import org.apache.asterix.lang.aql.parser.AQLParserFactory;
-import org.apache.asterix.lang.common.base.Expression;
-import org.apache.asterix.lang.common.base.Expression.Kind;
-import org.apache.asterix.lang.common.base.IParser;
-import org.apache.asterix.lang.common.base.IParserFactory;
-import org.apache.asterix.lang.common.base.Literal;
-import org.apache.asterix.lang.common.base.Statement;
-import org.apache.asterix.lang.common.expression.ListConstructor;
-import org.apache.asterix.lang.common.expression.LiteralExpr;
-import org.apache.asterix.lang.common.literal.DoubleLiteral;
-import org.apache.asterix.lang.common.literal.FloatLiteral;
-import org.apache.asterix.lang.common.literal.IntegerLiteral;
-import org.apache.asterix.lang.common.literal.LongIntegerLiteral;
-import org.apache.asterix.lang.common.literal.StringLiteral;
-import org.apache.asterix.lang.common.statement.Query;
-import org.apache.asterix.om.base.AMutableDouble;
-import org.apache.asterix.om.base.AMutableFloat;
-import org.apache.asterix.om.base.AMutableInt32;
-import org.apache.asterix.om.base.AMutableInt64;
-import org.apache.asterix.om.base.AMutableString;
-import org.apache.asterix.om.types.ATypeTag;
-import org.apache.asterix.om.types.BuiltinType;
-import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
-import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
-import org.apache.hyracks.api.exceptions.HyracksDataException;
-import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
-import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
-
-public abstract class RangeMapBuilder {
-    private static final IParserFactory parserFactory = new AQLParserFactory();
-
-    public static IRangeMap parseHint(Object hint) throws CompilationException {
-        ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
-        DataOutput out = abvs.getDataOutput();
-        abvs.reset();
-
-        IParser parser = parserFactory.createParser((String) hint);
-        List<Statement> hintStatements = parser.parse();
-        if (hintStatements.size() != 1) {
-            throw new CompilationException("Only one range statement is allowed for the range hint.");
-        }
-
-        // Translate the query into a Range Map
-        if (hintStatements.get(0).getKind() != Statement.Kind.QUERY) {
-            throw new CompilationException("Not a proper query for the range hint.");
-        }
-        Query q = (Query) hintStatements.get(0);
-
-        if (q.getBody().getKind() != Kind.LIST_CONSTRUCTOR_EXPRESSION) {
-            throw new CompilationException("The range hint must be a list.");
-        }
-        List<Expression> el = ((ListConstructor) q.getBody()).getExprList();
-        int offsets[] = new int[el.size()];
-
-        // Loop over list of literals
-        for (int i = 0; i < el.size(); ++i) {
-            Expression item = el.get(i);
-            if (item.getKind() == Kind.LITERAL_EXPRESSION) {
-                parseLiteralToBytes(item, out);
-                offsets[i] = abvs.getLength();
-            }
-            // TODO Add support for composite fields.
-        }
-
-        return new RangeMap(1, abvs.getByteArray(), offsets);
-    }
-
-    @SuppressWarnings("unchecked")
-    private static void parseLiteralToBytes(Expression item, DataOutput out) throws CompilationException {
-        AMutableDouble aDouble = new AMutableDouble(0);
-        AMutableFloat aFloat = new AMutableFloat(0);
-        AMutableInt64 aInt64 = new AMutableInt64(0);
-        AMutableInt32 aInt32 = new AMutableInt32(0);
-        AMutableString aString = new AMutableString("");
-        @SuppressWarnings("rawtypes")
-        ISerializerDeserializer serde;
-
-        Literal l = ((LiteralExpr) item).getValue();
-        try {
-            switch (l.getLiteralType()) {
-                case DOUBLE:
-                    DoubleLiteral dl = (DoubleLiteral) l;
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
-                    aDouble.setValue(dl.getValue());
-                    serde.serialize(aDouble, out);
-                    break;
-                case FLOAT:
-                    FloatLiteral fl = (FloatLiteral) l;
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
-                    aFloat.setValue(fl.getValue());
-                    serde.serialize(aFloat, out);
-                    break;
-                case INTEGER:
-                    IntegerLiteral il = (IntegerLiteral) l;
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
-                    aInt32.setValue(il.getValue());
-                    serde.serialize(aInt32, out);
-                    break;
-                case LONG:
-                    LongIntegerLiteral lil = (LongIntegerLiteral) l;
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
-                    aInt64.setValue(lil.getValue());
-                    serde.serialize(aInt64, out);
-                    break;
-                case STRING:
-                    StringLiteral sl = (StringLiteral) l;
-                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
-                    aString.setValue(sl.getValue());
-                    serde.serialize(aString, out);
-                    break;
-                default:
-                    throw new NotImplementedException("The range map builder has not been implemented for "
-                            + item.getKind() + " type of expressions.");
-            }
-        } catch (HyracksDataException e) {
-            throw new CompilationException(e.getMessage());
-        }
-    }
-
-    public static void verifyRangeOrder(IRangeMap rangeMap, boolean ascending) throws CompilationException {
-        // TODO Add support for composite fields.
-        int fieldIndex = 0;
-        int fieldType = rangeMap.getTag(0, 0);
-        BinaryComparatorFactoryProvider comparatorFactory = BinaryComparatorFactoryProvider.INSTANCE;
-        IBinaryComparatorFactory bcf =
-                comparatorFactory.getBinaryComparatorFactory(ATypeTag.VALUE_TYPE_MAPPING[fieldType], ascending);
-        IBinaryComparator comparator = bcf.createBinaryComparator();
-        int c = 0;
-        for (int split = 1; split < rangeMap.getSplitCount(); ++split) {
-            if (fieldType != rangeMap.getTag(fieldIndex, split)) {
-                throw new CompilationException("Range field contains more than a single type of items (" + fieldType
-                        + " and " + rangeMap.getTag(fieldIndex, split) + ").");
-            }
-            int previousSplit = split - 1;
-            try {
-                c = comparator.compare(rangeMap.getByteArray(fieldIndex, previousSplit),
-                        rangeMap.getStartOffset(fieldIndex, previousSplit),
-                        rangeMap.getLength(fieldIndex, previousSplit), rangeMap.getByteArray(fieldIndex, split),
-                        rangeMap.getStartOffset(fieldIndex, split), rangeMap.getLength(fieldIndex, split));
-            } catch (HyracksDataException e) {
-                throw new CompilationException(e);
-            }
-            if (c >= 0) {
-                throw new CompilationException("Range fields are not in sorted order.");
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
index 856073d..35c2ae8 100644
--- a/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
+++ b/asterixdb/asterix-lang-aql/src/main/javacc/AQL.jj
@@ -67,12 +67,12 @@ import org.apache.asterix.lang.aql.clause.DistinctClause;
 import org.apache.asterix.lang.aql.clause.ForClause;
 import org.apache.asterix.lang.aql.expression.FLWOGRExpression;
 import org.apache.asterix.lang.aql.expression.UnionExpr;
-import org.apache.asterix.lang.aql.util.RangeMapBuilder;
 import org.apache.asterix.lang.aql.util.AQLFormatPrintUtil;
 import org.apache.asterix.lang.common.base.Clause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.ILangExpression;
 import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
@@ -150,6 +150,7 @@ import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.RangeMapBuilder;
 import org.apache.asterix.metadata.utils.MetadataConstants;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
@@ -212,6 +213,10 @@ class AQLParser extends ScopeChecker implements IParser {
         return s.substring(1).trim();
     }
 
+    private static IParser createNewParser(String statement) {
+        return new AQLParser(statement);
+    }
+
     private static void checkBindingVariable(Expression returnExpression, VariableExpr var,
                                              ILangExpression bodyExpression) throws ParseException {
        if (returnExpression != null && var == null) {
@@ -2419,7 +2424,7 @@ Clause OrderbyClause()throws ParseException :
           }
           if (hint.startsWith(RANGE_HINT)) {
             try{
-              oc.setRangeMap(RangeMapBuilder.parseHint(hint.substring(RANGE_HINT.length())));
+              oc.setRangeMap(RangeMapBuilder.parseHint(createNewParser(hint.substring(RANGE_HINT.length()))));
             } catch (CompilationException e) {
               throw new ParseException(e.getMessage());
             }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-lang-common/pom.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/pom.xml b/asterixdb/asterix-lang-common/pom.xml
index 7d68b8b..dde41e0 100644
--- a/asterixdb/asterix-lang-common/pom.xml
+++ b/asterixdb/asterix-lang-common/pom.xml
@@ -103,5 +103,9 @@
       <groupId>org.apache.hyracks</groupId>
       <artifactId>hyracks-api</artifactId>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hyracks</groupId>
+      <artifactId>hyracks-data-std</artifactId>
+    </dependency>
   </dependencies>
 </project>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
index faefe1d..af8c725 100644
--- a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/clause/OrderbyClause.java
@@ -25,12 +25,12 @@ import org.apache.asterix.common.exceptions.CompilationException;
 import org.apache.asterix.lang.common.base.AbstractClause;
 import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.visitor.base.ILangVisitor;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
 public class OrderbyClause extends AbstractClause {
     private List<Expression> orderbyList;
     private List<OrderModifier> modifierList;
-    private IRangeMap rangeMap;
+    private RangeMap rangeMap; // can be null
     private int numFrames = -1;
     private int numTuples = -1;
 
@@ -90,17 +90,17 @@ public class OrderbyClause extends AbstractClause {
         this.numTuples = numTuples;
     }
 
-    public IRangeMap getRangeMap() {
+    public RangeMap getRangeMap() {
         return rangeMap;
     }
 
-    public void setRangeMap(IRangeMap rangeMap) {
+    public void setRangeMap(RangeMap rangeMap) {
         this.rangeMap = rangeMap;
     }
 
     @Override
     public int hashCode() {
-        return Objects.hash(modifierList, numFrames, numTuples, orderbyList);
+        return Objects.hash(modifierList, numFrames, numTuples, orderbyList, rangeMap);
     }
 
     @Override
@@ -113,6 +113,7 @@ public class OrderbyClause extends AbstractClause {
         }
         OrderbyClause target = (OrderbyClause) object;
         return Objects.equals(modifierList, target.modifierList) && numFrames == target.numFrames
-                && numTuples == target.numTuples && orderbyList.equals(target.orderbyList);
+                && numTuples == target.numTuples && orderbyList.equals(target.orderbyList)
+                && Objects.equals(rangeMap, target.rangeMap);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/RangeMapBuilder.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/RangeMapBuilder.java b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/RangeMapBuilder.java
new file mode 100644
index 0000000..c505c1c
--- /dev/null
+++ b/asterixdb/asterix-lang-common/src/main/java/org/apache/asterix/lang/common/util/RangeMapBuilder.java
@@ -0,0 +1,175 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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.lang.common.util;
+
+import java.io.DataOutput;
+import java.util.List;
+
+import org.apache.asterix.common.exceptions.CompilationException;
+import org.apache.asterix.common.exceptions.ErrorCode;
+import org.apache.asterix.formats.nontagged.BinaryComparatorFactoryProvider;
+import org.apache.asterix.formats.nontagged.SerializerDeserializerProvider;
+import org.apache.asterix.lang.common.base.Expression;
+import org.apache.asterix.lang.common.base.Expression.Kind;
+import org.apache.asterix.lang.common.base.IParser;
+import org.apache.asterix.lang.common.base.Literal;
+import org.apache.asterix.lang.common.base.Statement;
+import org.apache.asterix.lang.common.expression.ListConstructor;
+import org.apache.asterix.lang.common.expression.LiteralExpr;
+import org.apache.asterix.lang.common.literal.DoubleLiteral;
+import org.apache.asterix.lang.common.literal.FloatLiteral;
+import org.apache.asterix.lang.common.literal.IntegerLiteral;
+import org.apache.asterix.lang.common.literal.LongIntegerLiteral;
+import org.apache.asterix.lang.common.literal.StringLiteral;
+import org.apache.asterix.lang.common.statement.Query;
+import org.apache.asterix.om.base.AMutableDouble;
+import org.apache.asterix.om.base.AMutableFloat;
+import org.apache.asterix.om.base.AMutableInt32;
+import org.apache.asterix.om.base.AMutableInt64;
+import org.apache.asterix.om.base.AMutableString;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.BuiltinType;
+import org.apache.hyracks.algebricks.common.exceptions.NotImplementedException;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
+import org.apache.hyracks.api.dataflow.value.ISerializerDeserializer;
+import org.apache.hyracks.api.exceptions.HyracksDataException;
+import org.apache.hyracks.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+
+public class RangeMapBuilder {
+
+    private RangeMapBuilder() {
+    }
+
+    public static RangeMap parseHint(IParser parser) throws CompilationException {
+        ArrayBackedValueStorage abvs = new ArrayBackedValueStorage();
+        DataOutput out = abvs.getDataOutput();
+        abvs.reset();
+
+        List<Statement> hintStatements = parser.parse();
+        if (hintStatements.size() != 1) {
+            throw new CompilationException("Only one range statement is allowed for the range hint.");
+        }
+
+        // Translate the query into a Range Map
+        if (hintStatements.get(0).getKind() != Statement.Kind.QUERY) {
+            throw new CompilationException("Not a proper query for the range hint.");
+        }
+        Query q = (Query) hintStatements.get(0);
+
+        if (q.getBody().getKind() != Kind.LIST_CONSTRUCTOR_EXPRESSION) {
+            throw new CompilationException("The range hint must be a list.");
+        }
+        List<Expression> el = ((ListConstructor) q.getBody()).getExprList();
+        int[] offsets = new int[el.size()];
+
+        // Loop over list of literals
+        for (int i = 0; i < el.size(); ++i) {
+            Expression item = el.get(i);
+            if (item.getKind() == Kind.LITERAL_EXPRESSION) {
+                parseLiteralToBytes(item, out);
+                offsets[i] = abvs.getLength();
+            }
+            // TODO Add support for composite fields.
+        }
+
+        return new RangeMap(1, abvs.getByteArray(), offsets);
+    }
+
+    @SuppressWarnings("unchecked")
+    private static void parseLiteralToBytes(Expression item, DataOutput out) throws CompilationException {
+        AMutableDouble aDouble = new AMutableDouble(0);
+        AMutableFloat aFloat = new AMutableFloat(0);
+        AMutableInt64 aInt64 = new AMutableInt64(0);
+        AMutableInt32 aInt32 = new AMutableInt32(0);
+        AMutableString aString = new AMutableString("");
+        @SuppressWarnings("rawtypes")
+        ISerializerDeserializer serde;
+
+        Literal l = ((LiteralExpr) item).getValue();
+        try {
+            switch (l.getLiteralType()) {
+                case DOUBLE:
+                    DoubleLiteral dl = (DoubleLiteral) l;
+                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ADOUBLE);
+                    aDouble.setValue(dl.getValue());
+                    serde.serialize(aDouble, out);
+                    break;
+                case FLOAT:
+                    FloatLiteral fl = (FloatLiteral) l;
+                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AFLOAT);
+                    aFloat.setValue(fl.getValue());
+                    serde.serialize(aFloat, out);
+                    break;
+                case INTEGER:
+                    IntegerLiteral il = (IntegerLiteral) l;
+                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT32);
+                    aInt32.setValue(il.getValue());
+                    serde.serialize(aInt32, out);
+                    break;
+                case LONG:
+                    LongIntegerLiteral lil = (LongIntegerLiteral) l;
+                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.AINT64);
+                    aInt64.setValue(lil.getValue());
+                    serde.serialize(aInt64, out);
+                    break;
+                case STRING:
+                    StringLiteral sl = (StringLiteral) l;
+                    serde = SerializerDeserializerProvider.INSTANCE.getSerializerDeserializer(BuiltinType.ASTRING);
+                    aString.setValue(sl.getValue());
+                    serde.serialize(aString, out);
+                    break;
+                default:
+                    throw new NotImplementedException("The range map builder has not been implemented for "
+                            + item.getKind() + " type of expressions.");
+            }
+        } catch (HyracksDataException e) {
+            throw new CompilationException(ErrorCode.RANGE_MAP_ERROR, e, item.getSourceLocation(), e.getMessage());
+        }
+    }
+
+    public static void verifyRangeOrder(RangeMap rangeMap, boolean ascending) throws CompilationException {
+        // TODO Add support for composite fields.
+        int fieldIndex = 0;
+        int fieldType = rangeMap.getTag(0, 0);
+        BinaryComparatorFactoryProvider comparatorFactory = BinaryComparatorFactoryProvider.INSTANCE;
+        IBinaryComparatorFactory bcf =
+                comparatorFactory.getBinaryComparatorFactory(ATypeTag.VALUE_TYPE_MAPPING[fieldType], ascending);
+        IBinaryComparator comparator = bcf.createBinaryComparator();
+        int c = 0;
+        for (int split = 1; split < rangeMap.getSplitCount(); ++split) {
+            if (fieldType != rangeMap.getTag(fieldIndex, split)) {
+                throw new CompilationException("Range field contains more than a single type of items (" + fieldType
+                        + " and " + rangeMap.getTag(fieldIndex, split) + ").");
+            }
+            int previousSplit = split - 1;
+            try {
+                c = comparator.compare(rangeMap.getByteArray(), rangeMap.getStartOffset(fieldIndex, previousSplit),
+                        rangeMap.getLength(fieldIndex, previousSplit), rangeMap.getByteArray(),
+                        rangeMap.getStartOffset(fieldIndex, split), rangeMap.getLength(fieldIndex, split));
+            } catch (HyracksDataException e) {
+                throw new CompilationException(e);
+            }
+            if (c >= 0) {
+                throw new CompilationException("Range fields are not in sorted order.");
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
index e2a8759..13d1f8d 100644
--- a/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
+++ b/asterixdb/asterix-lang-sqlpp/src/main/javacc/SQLPP.jj
@@ -73,6 +73,7 @@ import org.apache.asterix.lang.common.base.Expression;
 import org.apache.asterix.lang.common.base.Literal;
 import org.apache.asterix.lang.common.base.IParser;
 import org.apache.asterix.lang.common.base.ILangExpression;
+import org.apache.asterix.lang.common.base.IParserFactory;
 import org.apache.asterix.lang.common.base.Statement;
 import org.apache.asterix.lang.common.clause.GroupbyClause;
 import org.apache.asterix.lang.common.clause.LetClause;
@@ -149,6 +150,7 @@ import org.apache.asterix.lang.common.struct.Identifier;
 import org.apache.asterix.lang.common.struct.OperatorType;
 import org.apache.asterix.lang.common.struct.QuantifiedPair;
 import org.apache.asterix.lang.common.struct.VarIdentifier;
+import org.apache.asterix.lang.common.util.RangeMapBuilder;
 import org.apache.asterix.lang.sqlpp.clause.AbstractBinaryCorrelateClause;
 import org.apache.asterix.lang.sqlpp.clause.FromClause;
 import org.apache.asterix.lang.sqlpp.clause.FromTerm;
@@ -200,7 +202,6 @@ class SQLPPParser extends ScopeChecker implements IParser {
     private static final String SKIP_SECONDARY_INDEX_SEARCH_HINT = "skip-index";
     private static final String VAL_FILE_HINT = "val-files";
     private static final String VAL_FILE_SAME_INDEX_HINT = "val-file-same-idx";
-
     private static final String GEN_FIELDS_HINT = "gen-fields";
 
     // data generator hints
@@ -241,6 +242,10 @@ class SQLPPParser extends ScopeChecker implements IParser {
         return s.substring(1).trim();
     }
 
+    private static IParser createNewParser(String statement) {
+        return new SQLPPParser(statement);
+    }
+
     private Token getHintToken(Token t) {
         return t.specialToken;
     }
@@ -3062,6 +3067,13 @@ OrderbyClause OrderbyClause() throws ParseException :
             oc.setNumFrames(numFrames);
             oc.setNumTuples(numTuples);
           }
+          if (hint.startsWith(RANGE_HINT)) {
+            try {
+              oc.setRangeMap(RangeMapBuilder.parseHint(createNewParser(hint.substring(RANGE_HINT.length()))));
+            } catch (CompilationException e) {
+              throw new SqlppParseException(getSourceLocation(getHintToken(token)), e.getMessage());
+            }
+          }
         }
       }
     <BY> orderbyExpr = Expression()

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/functions/BuiltinFunctions.java
----------------------------------------------------------------------
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 910c159..1ff6e57 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
@@ -103,6 +103,7 @@ import org.apache.asterix.om.typecomputer.impl.OrderedListOfAIntervalTypeCompute
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAPointTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAStringTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.OrderedListOfAnyTypeComputer;
+import org.apache.asterix.om.typecomputer.impl.ListOfSamplesTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.PropagateTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordAddFieldsTypeComputer;
 import org.apache.asterix.om.typecomputer.impl.RecordMergeTypeComputer;
@@ -477,6 +478,10 @@ public class BuiltinFunctions {
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-intermediate-stddev", 1);
     public static final FunctionIdentifier LOCAL_STDDEV =
             new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-stddev", 1);
+    public static final FunctionIdentifier LOCAL_SAMPLING =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-local-sampling", FunctionIdentifier.VARARGS);
+    public static final FunctionIdentifier RANGE_MAP =
+            new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "agg-range-map", 1);
 
     public static final FunctionIdentifier SCALAR_AVG = new FunctionIdentifier(FunctionConstants.ASTERIX_NS, "avg", 1);
     public static final FunctionIdentifier SCALAR_COUNT =
@@ -1381,6 +1386,8 @@ public class BuiltinFunctions {
         addPrivateFunction(LOCAL_STDDEV, LocalSingleVarStatisticsTypeComputer.INSTANCE, true);
         addFunction(STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(GLOBAL_STDDEV, NullableDoubleTypeComputer.INSTANCE, true);
+        addPrivateFunction(LOCAL_SAMPLING, ListOfSamplesTypeComputer.INSTANCE, true);
+        addPrivateFunction(RANGE_MAP, ABinaryTypeComputer.INSTANCE, true);
 
         addPrivateFunction(SERIAL_SQL_AVG, NullableDoubleTypeComputer.INSTANCE, true);
         addPrivateFunction(SERIAL_SQL_COUNT, AInt64TypeComputer.INSTANCE, true);
@@ -1832,6 +1839,14 @@ public class BuiltinFunctions {
 
         addScalarAgg(FIRST_ELEMENT, SCALAR_FIRST_ELEMENT);
 
+        // RANGE_MAP
+        addAgg(RANGE_MAP);
+        addAgg(LOCAL_SAMPLING);
+        addLocalAgg(RANGE_MAP, LOCAL_SAMPLING);
+        addIntermediateAgg(LOCAL_SAMPLING, RANGE_MAP);
+        addIntermediateAgg(RANGE_MAP, RANGE_MAP);
+        addGlobalAgg(RANGE_MAP, RANGE_MAP);
+
         // MIN
 
         addAgg(MIN);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListOfSamplesTypeComputer.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListOfSamplesTypeComputer.java b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListOfSamplesTypeComputer.java
new file mode 100644
index 0000000..1ae72e4
--- /dev/null
+++ b/asterixdb/asterix-om/src/main/java/org/apache/asterix/om/typecomputer/impl/ListOfSamplesTypeComputer.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.om.typecomputer.impl;
+
+import org.apache.asterix.om.pointables.base.DefaultOpenFieldType;
+import org.apache.asterix.om.typecomputer.base.AbstractResultTypeComputer;
+import org.apache.asterix.om.types.AOrderedListType;
+import org.apache.asterix.om.types.IAType;
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+
+/**
+ * List of samples type: [[ANY], [ANY],...]. Each inner list constitutes one sample. Inside the inner list (the sample),
+ * each item (or field) has its type tag.
+ */
+public class ListOfSamplesTypeComputer extends AbstractResultTypeComputer {
+
+    public static final AOrderedListType TYPE =
+            new AOrderedListType(DefaultOpenFieldType.NESTED_OPEN_AORDERED_LIST_TYPE, null);
+    public static final ListOfSamplesTypeComputer INSTANCE = new ListOfSamplesTypeComputer();
+
+    private ListOfSamplesTypeComputer() {
+    }
+
+    @Override
+    protected IAType getResultType(ILogicalExpression expr, IAType... strippedInputTypes) throws AlgebricksException {
+        return TYPE;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSamplingAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSamplingAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSamplingAggregateDescriptor.java
new file mode 100644
index 0000000..55d381d
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/LocalSamplingAggregateDescriptor.java
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 java.io.IOException;
+
+import org.apache.asterix.builders.IAsterixListBuilder;
+import org.apache.asterix.builders.OrderedListBuilder;
+import org.apache.asterix.common.config.CompilerProperties;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.typecomputer.impl.ListOfSamplesTypeComputer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.om.types.AbstractCollectionType;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+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.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+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.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+
+public class LocalSamplingAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private int numSamples;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new LocalSamplingAggregateDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_NUM_SAMPLES;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.LOCAL_SAMPLING;
+    }
+
+    @Override
+    public void setImmutableStates(Object... states) {
+        numSamples = (int) states[0];
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new LocalSamplingAggregateFunction(args, ctx, numSamples);
+            }
+        };
+    }
+
+    private class LocalSamplingAggregateFunction implements IAggregateEvaluator {
+        private final int numSamplesRequired;
+        private final ArrayBackedValueStorage storage;
+        private final IAsterixListBuilder listOfSamplesBuilder;
+        private final IAsterixListBuilder oneSampleBuilder;
+        private final IScalarEvaluator[] sampledFieldsEval;
+        private final IPointable inputFieldValue;
+        private int numSamplesTaken;
+
+        /**
+         * @param args the fields that constitute a sample, e.g., $$1, $$2
+         * @param context Hyracks task
+         * @throws HyracksDataException
+         */
+        private LocalSamplingAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+                int numSamplesRequired) throws HyracksDataException {
+            storage = new ArrayBackedValueStorage();
+            inputFieldValue = new VoidPointable();
+            sampledFieldsEval = new IScalarEvaluator[args.length];
+            for (int i = 0; i < args.length; i++) {
+                sampledFieldsEval[i] = args[i].createScalarEvaluator(context);
+            }
+            oneSampleBuilder = new OrderedListBuilder();
+            listOfSamplesBuilder = new OrderedListBuilder();
+            listOfSamplesBuilder.reset(ListOfSamplesTypeComputer.TYPE);
+            this.numSamplesRequired = numSamplesRequired > 0 ? numSamplesRequired
+                    : (int) CompilerProperties.Option.COMPILER_SORT_SAMPLES.defaultValue();
+        }
+
+        @Override
+        public void init() throws HyracksDataException {
+            numSamplesTaken = 0;
+            listOfSamplesBuilder.reset(ListOfSamplesTypeComputer.TYPE);
+        }
+
+        /**
+         * Receives data stream one tuple at a time from a data source and records samples.
+         * @param tuple one sample
+         * @throws HyracksDataException
+         */
+        @Override
+        public void step(IFrameTupleReference tuple) throws HyracksDataException {
+            if (numSamplesTaken >= numSamplesRequired) {
+                return;
+            }
+            // start over for a new sample
+            oneSampleBuilder.reset((AbstractCollectionType) ListOfSamplesTypeComputer.TYPE.getItemType());
+
+            for (IScalarEvaluator fieldEval : sampledFieldsEval) {
+                // add fields to make up one sample
+                fieldEval.evaluate(tuple, inputFieldValue);
+                oneSampleBuilder.addItem(inputFieldValue);
+            }
+            // prepare the sample to add it to the list of samples
+            storage.reset();
+            oneSampleBuilder.write(storage.getDataOutput(), true);
+            listOfSamplesBuilder.addItem(storage);
+            numSamplesTaken++;
+        }
+
+        /**
+         * Sends the list of samples to the global range-map generator.
+         * @param result list of samples
+         * @throws HyracksDataException
+         */
+        @Override
+        public void finish(IPointable result) throws HyracksDataException {
+            storage.reset();
+            if (numSamplesTaken == 0) {
+                // empty partition? then send system null as an indication of empty partition.
+                try {
+                    storage.getDataOutput().writeByte(ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG);
+                    result.set(storage);
+                } catch (IOException e) {
+                    throw HyracksDataException.create(e);
+                }
+            } else {
+                listOfSamplesBuilder.write(storage.getDataOutput(), true);
+                result.set(storage);
+            }
+        }
+
+        @Override
+        public void finishPartial(IPointable result) throws HyracksDataException {
+            finish(result);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/RangeMapAggregateDescriptor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/RangeMapAggregateDescriptor.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/RangeMapAggregateDescriptor.java
new file mode 100644
index 0000000..c967a94
--- /dev/null
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/aggregates/std/RangeMapAggregateDescriptor.java
@@ -0,0 +1,317 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectAscBinaryComparatorFactory;
+import org.apache.asterix.dataflow.data.nontagged.comparators.AObjectDescBinaryComparatorFactory;
+import org.apache.asterix.om.functions.BuiltinFunctions;
+import org.apache.asterix.om.functions.IFunctionDescriptor;
+import org.apache.asterix.om.functions.IFunctionDescriptorFactory;
+import org.apache.asterix.om.functions.IFunctionTypeInferer;
+import org.apache.asterix.om.types.ATypeTag;
+import org.apache.asterix.runtime.aggregates.base.AbstractAggregateFunctionDynamicDescriptor;
+import org.apache.asterix.runtime.evaluators.common.ListAccessor;
+import org.apache.asterix.runtime.functions.FunctionTypeInferers;
+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.IScalarEvaluator;
+import org.apache.hyracks.algebricks.runtime.base.IScalarEvaluatorFactory;
+import org.apache.hyracks.api.context.IHyracksTaskContext;
+import org.apache.hyracks.api.dataflow.value.IBinaryComparator;
+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.data.std.util.ArrayBackedValueStorage;
+import org.apache.hyracks.dataflow.common.data.accessors.IFrameTupleReference;
+import org.apache.hyracks.dataflow.common.data.marshalling.ByteArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntArraySerializerDeserializer;
+import org.apache.hyracks.dataflow.common.data.marshalling.IntegerSerializerDeserializer;
+
+/**
+ * <pre>
+ * {@see {@link org.apache.hyracks.dataflow.common.data.partition.range.RangeMap}} for some description of the range map
+ * structure that is produced by this function. Given a list of samples and a number of partitions "k", the algorithm
+ * of this function operates as follows (s = sample):
+ * It picks (k - 1) split points out of the samples by dividing num_samples/num_partitions. For 4 partitions, it's 3:
+ * s0,s1,s2,s3,s4,s5,s6,s7,s8,s9,s10,s11,s12,s13,s14,s15; 16/4 = 4; range map = [s3, s7, s11]
+ *           |           |             |
+ *
+ * s0,s1,s2,s3,s4,s5,s6; 7/4 = 2; range map = [s1, s3, s5]
+ *     |     |     |
+ *
+ * s0,s1,s2,s3,s4; 5/4 = 2; range map = [s1, s3, s4]; if we go out of bound for the last split, we pick the last item.
+ *     |     |  |
+ *
+ * s0,s1,s2,s3; if #_samples <= #_partitions, we sweep from the beginning (should be rare). range map = [s0, s1, s2]
+ *  |  |  |
+ *
+ * s0,s1; if there are way less samples, we sweep and repeat the last item; range map = [s0, s1, s1];
+ * Note: a sample (and therefore also a split point) could be single-column or multi-column.
+ * </pre>
+ */
+public class RangeMapAggregateDescriptor extends AbstractAggregateFunctionDynamicDescriptor {
+    private static final long serialVersionUID = 1L;
+    private boolean[] ascendingFlags;
+    private int numOfPartitions;
+    private int numOrderFields;
+
+    public static final IFunctionDescriptorFactory FACTORY = new IFunctionDescriptorFactory() {
+        @Override
+        public IFunctionDescriptor createFunctionDescriptor() {
+            return new RangeMapAggregateDescriptor();
+        }
+
+        @Override
+        public IFunctionTypeInferer createFunctionTypeInferer() {
+            return FunctionTypeInferers.SET_SORTING_PARAMETERS;
+        }
+    };
+
+    @Override
+    public FunctionIdentifier getIdentifier() {
+        return BuiltinFunctions.RANGE_MAP;
+    }
+
+    /**
+     * The sampling function, which generates the splitting vector, needs to know the number of partitions in order to
+     * determine how many split points to pick out of the samples. It also needs to know the ascending/descending of
+     * each sort field so that it can sort the samples accordingly first and then pick the (number of partitions - 1)
+     * split points out of the sorted samples.
+     * @param states states[0]: number of partitions, states[1]: ascending flags
+     */
+    @Override
+    public void setImmutableStates(Object... states) {
+        numOfPartitions = (int) states[0];
+        ascendingFlags = (boolean[]) states[1];
+        numOrderFields = ascendingFlags.length;
+    }
+
+    @Override
+    public IAggregateEvaluatorFactory createAggregateEvaluatorFactory(final IScalarEvaluatorFactory[] args) {
+        return new IAggregateEvaluatorFactory() {
+            private static final long serialVersionUID = 1L;
+
+            @Override
+            public IAggregateEvaluator createAggregateEvaluator(final IHyracksTaskContext ctx)
+                    throws HyracksDataException {
+                return new GlobalSamplingAggregateFunction(args, ctx, ascendingFlags, numOfPartitions, numOrderFields);
+            }
+        };
+    }
+
+    private class GlobalSamplingAggregateFunction implements IAggregateEvaluator {
+        private final IScalarEvaluator localSamplesEval;
+        private final IPointable localSamples;
+        private final List<List<byte[]>> finalSamples;
+        private final Comparator<List<byte[]>> comparator;
+        private final int numOfPartitions;
+        private final int numOrderByFields;
+        private final ListAccessor listOfSamples;
+        private final ListAccessor oneSample;
+        private final IPointable oneSamplePointable;
+        private final ArrayBackedValueStorage oneSampleStorage;
+        private final IPointable field;
+        private final ArrayBackedValueStorage storage;
+
+        @SuppressWarnings("unchecked")
+        private GlobalSamplingAggregateFunction(IScalarEvaluatorFactory[] args, IHyracksTaskContext context,
+                boolean[] ascending, int numOfPartitions, int numOrderByFields) throws HyracksDataException {
+            localSamples = new VoidPointable();
+            localSamplesEval = args[0].createScalarEvaluator(context);
+            finalSamples = new ArrayList<>();
+            comparator = createComparator(ascending);
+            this.numOfPartitions = numOfPartitions;
+            this.numOrderByFields = numOrderByFields;
+            listOfSamples = new ListAccessor();
+            oneSample = new ListAccessor();
+            oneSamplePointable = new VoidPointable();
+            oneSampleStorage = new ArrayBackedValueStorage();
+            field = new VoidPointable();
+            storage = new ArrayBackedValueStorage();
+        }
+
+        @Override
+        public void init() throws HyracksDataException {
+            finalSamples.clear();
+        }
+
+        /**
+         * Receives the local samples and appends them to the final list of samples.
+         * @param tuple the partition's samples
+         * @throws HyracksDataException
+         */
+        @Override
+        public void step(IFrameTupleReference tuple) throws HyracksDataException {
+            // check if empty stream (system_null), i.e. partition is empty, so no samples
+            localSamplesEval.evaluate(tuple, localSamples);
+            byte tag = localSamples.getByteArray()[localSamples.getStartOffset()];
+            if (tag == ATypeTag.SERIALIZED_SYSTEM_NULL_TYPE_TAG) {
+                return;
+            }
+            // deserialize the samples received from the local partition
+            listOfSamples.reset(localSamples.getByteArray(), localSamples.getStartOffset());
+            int numberOfSamples = listOfSamples.size();
+
+            // "sample" & "addedSample" are lists to support multi-column instead of one value, i.e. <3,"dept">
+            List<byte[]> addedSample;
+            int numberOfFields;
+            // add the samples to the final samples
+            try {
+                for (int i = 0; i < numberOfSamples; i++) {
+                    oneSampleStorage.reset();
+                    listOfSamples.getOrWriteItem(i, oneSamplePointable, oneSampleStorage);
+                    oneSample.reset(oneSamplePointable.getByteArray(), oneSamplePointable.getStartOffset());
+                    numberOfFields = oneSample.size();
+                    addedSample = new ArrayList<>(numberOfFields);
+                    for (int j = 0; j < numberOfFields; j++) {
+                        storage.reset();
+                        oneSample.getOrWriteItem(j, field, storage);
+                        addedSample.add(Arrays.copyOfRange(field.getByteArray(), field.getStartOffset(),
+                                field.getStartOffset() + field.getLength()));
+                    }
+                    finalSamples.add(addedSample);
+                }
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+        }
+
+        /**
+         * Produces the range map out of the collected samples from each partition. The final list of samples is sorted
+         * first. Then, we select the split points by dividing the samples evenly.
+         * @param result contains the serialized range map.
+         * @throws HyracksDataException
+         */
+        @Override
+        public void finish(IPointable result) throws HyracksDataException {
+            // storage == all serialized split values of all split points
+            storage.reset();
+            DataOutput allSplitValuesOut = storage.getDataOutput();
+            int[] endOffsets;
+            try {
+                // check if empty dataset, i.e. no samples have been received from any partition
+                if (finalSamples.isEmpty()) {
+                    // a range map with null values
+                    endOffsets = new int[numOrderByFields];
+                    for (int sortField = 0; sortField < numOrderByFields; sortField++) {
+                        allSplitValuesOut.write(ATypeTag.SERIALIZED_NULL_TYPE_TAG);
+                        endOffsets[sortField] = storage.getLength();
+                    }
+                } else {
+                    finalSamples.sort(comparator);
+                    // divide the samples evenly and pick the boundaries as split points
+                    int nextSplitOffset = (int) Math.ceil(finalSamples.size() / (double) numOfPartitions);
+                    int nextSplitIndex = nextSplitOffset - 1;
+                    int endOffsetsCounter = 0;
+                    int numRequiredSplits = numOfPartitions - 1;
+                    endOffsets = new int[numRequiredSplits * numOrderByFields];
+                    List<byte[]> sample;
+                    for (int split = 1; split <= numRequiredSplits; split++) {
+                        // pick the split point from sorted samples (could be <3> or <4,"John"> if it's multi-column)
+                        sample = finalSamples.get(nextSplitIndex);
+                        for (int column = 0; column < sample.size(); column++) {
+                            allSplitValuesOut.write(sample.get(column));
+                            endOffsets[endOffsetsCounter++] = storage.getLength();
+                        }
+                        // go to the next split point
+                        nextSplitIndex += nextSplitOffset;
+                        // in case we go beyond the boundary of samples, we pick the last sample repeatedly
+                        if (nextSplitIndex >= finalSamples.size()) {
+                            nextSplitIndex = finalSamples.size() - 1;
+                        }
+                    }
+                }
+            } catch (IOException e) {
+                throw HyracksDataException.create(e);
+            }
+
+            serializeRangemap(numOrderByFields, storage.getByteArray(), endOffsets, result);
+        }
+
+        @Override
+        public void finishPartial(IPointable result) throws HyracksDataException {
+            finish(result);
+        }
+
+        /**
+         * Creates the comparator that sorts all the collected samples before picking split points.
+         * @param ascending ascending or descending flag for each sort column.
+         * @return the described comparator
+         */
+        private Comparator<List<byte[]>> createComparator(boolean[] ascending) {
+            // create the generic comparator for each sort field
+            IBinaryComparator[] fieldsComparators = new IBinaryComparator[ascending.length];
+            for (int i = 0; i < ascending.length; i++) {
+                if (ascending[i]) {
+                    fieldsComparators[i] = AObjectAscBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+                } else {
+                    fieldsComparators[i] = AObjectDescBinaryComparatorFactory.INSTANCE.createBinaryComparator();
+                }
+            }
+
+            return (splitPoint1, splitPoint2) -> {
+                try {
+                    // two split points must have the same num of fields
+                    int numFields = splitPoint1.size();
+                    int result = 0;
+                    byte[] field1;
+                    byte[] field2;
+                    for (int i = 0; i < numFields; i++) {
+                        field1 = splitPoint1.get(i);
+                        field2 = splitPoint2.get(i);
+                        result = fieldsComparators[i].compare(field1, 0, field1.length, field2, 0, field2.length);
+                        if (result != 0) {
+                            return result;
+                        }
+                    }
+                    return result;
+                } catch (HyracksDataException e) {
+                    throw new IllegalStateException(e);
+                }
+            };
+        }
+
+        /**
+         * Serializes the range map object defined by the below attributes into the "result". The range map object is
+         * serialized as binary data.
+         * @param numberFields the number of order-by fields (the sort fields)
+         * @param splitValues the serialized split values stored one after the other
+         * @param endOffsets the end offsets of each split value
+         * @param result where the range map object is serialized
+         * @throws HyracksDataException
+         */
+        private void serializeRangemap(int numberFields, byte[] splitValues, int[] endOffsets, IPointable result)
+                throws HyracksDataException {
+            ArrayBackedValueStorage serRangeMap = new ArrayBackedValueStorage();
+            IntegerSerializerDeserializer.write(numberFields, serRangeMap.getDataOutput());
+            ByteArraySerializerDeserializer.write(splitValues, serRangeMap.getDataOutput());
+            IntArraySerializerDeserializer.write(endOffsets, serRangeMap.getDataOutput());
+
+            result.set(serRangeMap.getByteArray(), serRangeMap.getStartOffset(), serRangeMap.getLength());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionCollection.java
----------------------------------------------------------------------
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 9541d69..5affbca 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
@@ -68,6 +68,7 @@ import org.apache.asterix.runtime.aggregates.serializable.std.SerializableSumAgg
 import org.apache.asterix.runtime.aggregates.std.AvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.CountAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalAvgAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.RangeMapAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalSqlStddevAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.GlobalStddevAggregateDescriptor;
@@ -78,6 +79,7 @@ import org.apache.asterix.runtime.aggregates.std.IntermediateStddevAggregateDesc
 import org.apache.asterix.runtime.aggregates.std.LocalAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalMaxAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalMinAggregateDescriptor;
+import org.apache.asterix.runtime.aggregates.std.LocalSamplingAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlAvgAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlMaxAggregateDescriptor;
 import org.apache.asterix.runtime.aggregates.std.LocalSqlMinAggregateDescriptor;
@@ -458,6 +460,8 @@ public final class FunctionCollection implements IFunctionCollection {
         fc.add(LocalStddevAggregateDescriptor.FACTORY);
         fc.add(IntermediateStddevAggregateDescriptor.FACTORY);
         fc.add(GlobalStddevAggregateDescriptor.FACTORY);
+        fc.add(LocalSamplingAggregateDescriptor.FACTORY);
+        fc.add(RangeMapAggregateDescriptor.FACTORY);
 
         // serializable aggregates
         fc.add(SerializableCountAggregateDescriptor.FACTORY);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
index b9c58c7..44e3eb7 100644
--- a/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
+++ b/asterixdb/asterix-runtime/src/main/java/org/apache/asterix/runtime/functions/FunctionTypeInferers.java
@@ -19,6 +19,9 @@
 
 package org.apache.asterix.runtime.functions;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.asterix.common.config.CompilerProperties;
 import org.apache.asterix.om.base.AOrderedList;
 import org.apache.asterix.om.base.AString;
@@ -42,9 +45,6 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCa
 import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
 
-import java.util.ArrayList;
-import java.util.List;
-
 /**
  * Implementations of {@link IFunctionTypeInferer} for built-in functions
  */
@@ -95,6 +95,26 @@ public final class FunctionTypeInferers {
         }
     };
 
+    public static final IFunctionTypeInferer SET_SORTING_PARAMETERS = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression funCallExpr = (AbstractFunctionCallExpression) expr;
+            Object[] sortingParameters = funCallExpr.getOpaqueParameters();
+            fd.setImmutableStates(sortingParameters[0], sortingParameters[1]);
+        }
+    };
+
+    public static final IFunctionTypeInferer SET_NUM_SAMPLES = new IFunctionTypeInferer() {
+        @Override
+        public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,
+                CompilerProperties compilerProps) throws AlgebricksException {
+            AbstractFunctionCallExpression funCallExpr = (AbstractFunctionCallExpression) expr;
+            Object[] samplingParameters = funCallExpr.getOpaqueParameters();
+            fd.setImmutableStates(samplingParameters[0]);
+        }
+    };
+
     public static final class CastTypeInferer implements IFunctionTypeInferer {
         @Override
         public void infer(ILogicalExpression expr, IFunctionDescriptor fd, IVariableTypeEnvironment context,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
index 2d13baf..3794328 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/LogicalOperatorTag.java
@@ -28,6 +28,7 @@ public enum LogicalOperatorTag {
     EXCHANGE,
     DELEGATE_OPERATOR,
     EXTERNAL_LOOKUP,
+    FORWARD,
     GROUP,
     INDEX_INSERT_DELETE_UPSERT,
     INNERJOIN,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
index fc702ce..cbe0882 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/OperatorAnnotations.java
@@ -22,7 +22,8 @@ public interface OperatorAnnotations {
     // hints
     public static final String USE_HASH_GROUP_BY = "USE_HASH_GROUP_BY"; // -->
     public static final String USE_EXTERNAL_GROUP_BY = "USE_EXTERNAL_GROUP_BY"; // -->
-    public static final String USE_RANGE_CONNECTOR = "USE_RANGE_CONNECTOR"; // -->
+    public static final String USE_STATIC_RANGE = "USE_STATIC_RANGE"; // -->
+    public static final String USE_DYNAMIC_RANGE = "USE_DYNAMIC_RANGE";
     // Boolean
     public static final String CARDINALITY = "CARDINALITY"; // -->
     // Integer

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
index db9728b..ac1de5a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/base/PhysicalOperatorTag.java
@@ -30,6 +30,7 @@ public enum PhysicalOperatorTag {
     DELEGATE_OPERATOR,
     EXTERNAL_GROUP_BY,
     EXTERNAL_LOOKUP,
+    FORWARD,
     HASH_GROUP_BY,
     HASH_PARTITION_EXCHANGE,
     HASH_PARTITION_MERGE_EXCHANGE,
@@ -54,6 +55,7 @@ public enum PhysicalOperatorTag {
     RANDOM_MERGE_EXCHANGE,
     RANGE_PARTITION_EXCHANGE,
     RANGE_PARTITION_MERGE_EXCHANGE,
+    SEQUENTIAL_MERGE_EXCHANGE,
     REPLICATE,
     RTREE_SEARCH,
     RUNNING_AGGREGATE,

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
index 80d6f95..1cc206f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AbstractFunctionCallExpression.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.core.algebra.expressions;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedList;
@@ -205,19 +206,7 @@ public abstract class AbstractFunctionCallExpression extends AbstractLogicalExpr
                     return false;
                 }
             }
-            if (opaqueParameters != null) {
-                if (opaqueParameters.length != fce.opaqueParameters.length) {
-                    return false;
-                }
-                for (int i = 0; i < opaqueParameters.length; i++) {
-                    Object opaqueParameter = opaqueParameters[i];
-                    Object fceOpaqueParameter = fce.opaqueParameters[i];
-                    if (!opaqueParameter.equals(fceOpaqueParameter)) {
-                        return false;
-                    }
-                }
-            }
-            return true;
+            return Arrays.deepEquals(opaqueParameters, fce.opaqueParameters);
         }
     }
 
@@ -228,9 +217,7 @@ public abstract class AbstractFunctionCallExpression extends AbstractLogicalExpr
             h = h * 41 + e.getValue().hashCode();
         }
         if (opaqueParameters != null) {
-            for (int i = 0; i < opaqueParameters.length; i++) {
-                h = h * 31 + opaqueParameters[i].hashCode();
-            }
+            h = h * 31 + Arrays.deepHashCode(opaqueParameters);
         }
         return h;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
index bdd820e..97c4252 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/expressions/AggregateFunctionCallExpression.java
@@ -60,6 +60,8 @@ public class AggregateFunctionCallExpression extends AbstractFunctionCallExpress
         fun.setStepTwoAggregate(stepTwoAggregate);
         fun.setStepOneAggregate(stepOneAggregate);
         fun.setSourceLocation(sourceLoc);
+        // opaqueParameters are not really cloned
+        fun.setOpaqueParameters(getOpaqueParameters());
         return fun;
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java
index 3bb0f47..62b6a2d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/AbstractReplicateOperator.java
@@ -23,6 +23,7 @@ import java.util.List;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
@@ -39,7 +40,7 @@ import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionRef
 public abstract class AbstractReplicateOperator extends AbstractLogicalOperator {
 
     private int outputArity;
-    protected boolean[] outputMaterializationFlags;
+    private boolean[] outputMaterializationFlags;
     private List<Mutable<ILogicalOperator>> outputs;
 
     public AbstractReplicateOperator(int outputArity) {
@@ -95,6 +96,19 @@ public abstract class AbstractReplicateOperator extends AbstractLogicalOperator
         return outputs;
     }
 
+    public void setOutputs(List<Pair<Mutable<ILogicalOperator>, Boolean>> newOutputs) {
+        // shrinking or expanding num of outputs
+        if (outputMaterializationFlags.length != newOutputs.size()) {
+            outputMaterializationFlags = new boolean[newOutputs.size()];
+        }
+        outputs.clear();
+        for (int i = 0; i < newOutputs.size(); i++) {
+            outputs.add(newOutputs.get(i).first);
+            outputMaterializationFlags[i] = newOutputs.get(i).second;
+        }
+        outputArity = newOutputs.size();
+    }
+
     @Override
     public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
         return createPropagatingAllInputsTypeEnvironment(ctx);


[14/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..6ff2729
--- /dev/null
+++ 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
@@ -0,0 +1,358 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$80(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$65]  |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|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- REPLICATE  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$66][$$idRight_0]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ASSIGN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                                -- EXTERNAL_GROUP_BY[$$162, $$160]  |PARTITIONED|
+                                                        {
+                                                          -- AGGREGATE  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                        }
+                                                  -- HASH_PARTITION_EXCHANGE [$$162, $$160]  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- STREAM_SELECT  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                                -- UNNEST  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- PRE_CLUSTERED_GROUP_BY[$$93]  |PARTITIONED|
+                                                                            {
+                                                                              -- AGGREGATE  |LOCAL|
+                                                                                -- STREAM_SELECT  |LOCAL|
+                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                            }
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- STABLE_SORT [$$93(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- UNNEST  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- REPLICATE  |PARTITIONED|
+                                                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                      -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                  {
+                                                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                  }
+                                                                                                            -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                              -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                      {
+                                                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                      }
+                                                                                                                -- 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  |PARTITIONED|
+                                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                              -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- UNNEST  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- PRE_CLUSTERED_GROUP_BY[$$87]  |PARTITIONED|
+                                                                                {
+                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                }
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- STABLE_SORT [$$87(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- UNNEST  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                    -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                {
+                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                }
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                            -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                    {
+                                                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                    }
+                                                                                                              -- 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  |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[$$65]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$65]  |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|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- REPLICATE  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- HYBRID_HASH_JOIN [$$66][$$idRight_0]  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                                          -- EXTERNAL_GROUP_BY[$$162, $$160]  |PARTITIONED|
+                                                                  {
+                                                                    -- AGGREGATE  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                  }
+                                                            -- HASH_PARTITION_EXCHANGE [$$162, $$160]  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- STREAM_SELECT  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- PRE_CLUSTERED_GROUP_BY[$$93]  |PARTITIONED|
+                                                                                      {
+                                                                                        -- AGGREGATE  |LOCAL|
+                                                                                          -- STREAM_SELECT  |LOCAL|
+                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                      }
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STABLE_SORT [$$93(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- UNNEST  |PARTITIONED|
+                                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                    -- REPLICATE  |PARTITIONED|
+                                                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                                          -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                              -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                                -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                    -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                            {
+                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                            }
+                                                                                                                      -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                                        -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                                {
+                                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                                }
+                                                                                                                          -- 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  |PARTITIONED|
+                                                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                        -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- UNNEST  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- PRE_CLUSTERED_GROUP_BY[$$87]  |PARTITIONED|
+                                                                                          {
+                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                              -- STREAM_SELECT  |LOCAL|
+                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                          }
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STABLE_SORT [$$87(ASC), $$i(ASC)]  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- UNNEST  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                              -- REPLICATE  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- SORT_MERGE_EXCHANGE [$$167(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                              -- STABLE_SORT [$$167(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EXTERNAL_GROUP_BY[$$174]  |PARTITIONED|
+                                                                                                                          {
+                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                          }
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$174]  |PARTITIONED|
+                                                                                                                      -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                              {
+                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                              }
+                                                                                                                        -- 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  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..623c04d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
@@ -0,0 +1,90 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$68(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$68(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$55]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH  |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  |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[$$55]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- RTREE_SEARCH  |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  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..18c2244
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
@@ -0,0 +1,92 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |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  |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  |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|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |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  |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  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..77b3643
--- /dev/null
+++ 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
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |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|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..77b3643
--- /dev/null
+++ 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
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |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|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e2db3f6
--- /dev/null
+++ 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
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |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|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e2db3f6
--- /dev/null
+++ 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
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |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|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[20/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..0384667
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Mu')
+order by o.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..578ef74
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
new file mode 100644
index 0000000..eb17ed4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)[0]
+order by c.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
new file mode 100644
index 0000000..412ffdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)[0]
+order by c.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
new file mode 100644
index 0000000..44f6164
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 3)
+order by c.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..cf8b4a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-edit-distance_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.interests,['computers','wine','walking']) <= 1)
+order by c.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..0b8ec46
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+set `simthreshold` `1`;
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (c.interests ~= ['computers','wine','walking'])
+order by c.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..27157ff
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-basic/word-contains_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
new file mode 100644
index 0000000..3b861b1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],3)
+where ed[0]
+order by c.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
new file mode 100644
index 0000000..f5d9888
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerType as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create  dataset Customers(CustomerType) primary key cid;
+
+create  index interests_index  on Customers (interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.interests,['computers','wine','walking'],1)
+where ed[0]
+order by c.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..2c5ac71
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.`message-text`,t2.`message-text`,7)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..2348dd9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgKeywordIx  on TweetMessages (`message-text`) type keyword;
+
+create  index msgNgramIx  on TweetMessages (`message-text`) type ngram (3);
+
+create  index topicKeywordIx  on TweetMessages (`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`referred-topics`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`referred-topics`}
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t1.`referred-topics`,t2.`referred-topics`,0.500000f)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..5d49319
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/inverted-index-join/ngram-contains_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-core_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-core_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-core_ps.sqlpp
new file mode 100644
index 0000000..d8a4d6c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-core_ps.sqlpp
@@ -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.
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type test.DBLPType as closed {
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as closed {
+  id : bigint,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+create  dataset CSX(CSXType) primary key id;
+
+set `compiler.sort.parallel` "true";
+
+SELECT a.id AS aid, b.id AS bid
+FROM DBLP AS a
+LEFT OUTER UNNEST
+(
+  SELECT VALUE b
+  FROM CSX AS b
+  WHERE a.authors = b.authors
+) AS b
+ORDER BY a.id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-sugar_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-sugar_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-sugar_ps.sqlpp
new file mode 100644
index 0000000..ff38e33
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/loj-sugar_ps.sqlpp
@@ -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.
+ */
+drop  dataverse test if exists;
+create  dataverse test;
+use test;
+
+create type test.DBLPType as closed {
+  id : bigint,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.CSXType as closed {
+  id : bigint,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+create  dataset CSX(CSXType) primary key id;
+
+set `compiler.sort.parallel` "true";
+
+SELECT a.id AS aid, b.id AS bid
+FROM DBLP AS a
+LEFT OUTER JOIN CSX AS b
+ON a.authors = b.authors
+ORDER BY a.id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..bc0f28c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where (t1.nested.countA /*+ indexnl */  = t2.nested.countB)
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..3deb837
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where ((t1.nested.countA /*+ indexnl */  = t2.nested.countB) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..99a3497
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Mu')
+order by o.nested.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..4f07d69
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
new file mode 100644
index 0000000..a11fec7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],3)[0]
+order by c.nested.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
new file mode 100644
index 0000000..c9b6d13
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-check_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-check.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],1)[0]
+order by c.nested.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
new file mode 100644
index 0000000..db3a364
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.nested.interests,['computers','wine','walking']) <= 3)
+order by c.nested.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..05a8936
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-edit-distance_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the edit-distance function on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-edit-distance.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (test.`edit-distance`(c.nested.interests,['computers','wine','walking']) <= 1)
+order by c.nested.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
new file mode 100644
index 0000000..b53584e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query with ~= using edit-distance on lists.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_olist-fuzzyeq-edit-distance.adm";
+set `simfunction` `edit-distance`;
+set `simthreshold` `1`;
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+where (c.nested.interests ~= ['computers','wine','walking'])
+order by c.nested.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..e682634
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-basic/word-contains_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index keyword_index  on DBLP (nested.title) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
new file mode 100644
index 0000000..5d57ef6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],3)
+where ed[0]
+order by c.nested.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
new file mode 100644
index 0000000..acafb14
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the edit-distance-check function on lists.
+ *                  Tests that the optimizer rule correctly drills through the let clauses.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.AddressType as
+ closed {
+  number : integer,
+  street : string,
+  city : string
+};
+
+create type test.CustomerTypetmp as
+ closed {
+  cid : integer,
+  name : string,
+  age : integer?,
+  address : AddressType?,
+  interests : [string],
+  children : [{
+          name : string,
+          age : integer?
+      }
+]
+};
+
+create type test.CustomerType as
+ closed {
+  nested : CustomerTypetmp
+};
+
+create  dataset Customers(CustomerType) primary key nested.cid;
+
+create  dataset Customers2(CustomerType) primary key nested.cid;
+
+create  index interests_index  on Customers (nested.interests) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-complex_olist-edit-distance-check-let.adm";
+set `compiler.sort.parallel` "true";
+
+select element c
+from  Customers as c
+with  ed as test.`edit-distance-check`(c.nested.interests,['computers','wine','walking'],1)
+where ed[0]
+order by c.nested.cid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..bfd2bf4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgNgramIx  on TweetMessages (nested.`message-text`) type ngram (3);
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`message-text`},'similar-tweets':(
+        select element {'id':t2.nested.tweetid,'topics':t2.nested.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.nested.`message-text`,t2.nested.`message-text`,7)
+        where (sim[0] and (t2.nested.tweetid != t1.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid > test.bigint('240'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..292925a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+ closed {
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index topicKeywordIx  on TweetMessages (nested.`referred-topics`) type keyword;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-jaccard-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`referred-topics`},'similar-tweets':(
+        select element {'id':t2.nested.tweetid,'topics':t2.nested.`referred-topics`}
+        from  TweetMessages as t2
+        with  sim as test.`similarity-jaccard-check`(t1.nested.`referred-topics`,t2.nested.`referred-topics`,0.500000f)
+        where (sim[0] and (t2.nested.tweetid != t1.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid > test.bigint('240'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..245e3ac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.nested.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..2481d9e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -0,0 +1,82 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+ closed {
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`) type rtree;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.nested.`sender-location`,n) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;


[15/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..77b3643
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |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|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..e2db3f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
@@ -0,0 +1,86 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |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|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$54(ASC), $$55(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$54]  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$74(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(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|
+                            -- DATASOURCE_SCAN  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..9c29537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(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  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
new file mode 100644
index 0000000..575076b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(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|
+                            -- DATASOURCE_SCAN  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check_ps.plan
new file mode 100644
index 0000000..da301ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-check_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(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  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.plan
new file mode 100644
index 0000000..575076b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(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|
+                            -- DATASOURCE_SCAN  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance_ps.plan
new file mode 100644
index 0000000..da301ae
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-edit-distance_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$17(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  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
new file mode 100644
index 0000000..b4cb762
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$16(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$16(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  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$21(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/word-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(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|
+                            -- DATASOURCE_SCAN  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
new file mode 100644
index 0000000..19649ab
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |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_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- DATASOURCE_SCAN  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.plan
new file mode 100644
index 0000000..ae93a76
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-index/inverted-index-complex/olist-edit-distance-check-let_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |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_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- ASSIGN  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- BTREE_SEARCH  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$30(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..9ed007f
--- /dev/null
+++ 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
@@ -0,0 +1,190 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$80(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$86][$$65]  |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  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$65]  |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  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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  |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[$$65]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$86][$$65]  |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  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$65]  |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  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[13/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(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|
+                            -- DATASOURCE_SCAN  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..9c29537
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,42 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(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  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- BTREE_SEARCH  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- STABLE_SORT [$$20(ASC)]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/word-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..54584d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,32 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(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|
+                            -- DATASOURCE_SCAN  |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|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..9ed007f
--- /dev/null
+++ 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
@@ -0,0 +1,190 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$80(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$65]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$86][$$65]  |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  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$65]  |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  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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  |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[$$65]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$65(ASC), $$66(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$65]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$86][$$65]  |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  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- HASH_PARTITION_EXCHANGE [$$65]  |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  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STABLE_SORT [$$98(ASC)]  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- ASSIGN  |PARTITIONED|
+                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..428c860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_01_ps.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |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  |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|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |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  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..428c860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_02_ps.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |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  |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|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |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  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_03_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |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|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/inverted-index-join/ngram-contains_04_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |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|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..623c04d
--- /dev/null
+++ 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
@@ -0,0 +1,90 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$68(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$68(ASC)]  |PARTITIONED|
+                -- FORWARD  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- REPLICATE  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- PRE_CLUSTERED_GROUP_BY[$$55]  |PARTITIONED|
+                                {
+                                  -- AGGREGATE  |LOCAL|
+                                    -- STREAM_SELECT  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                }
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- RTREE_SEARCH  |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  |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[$$55]  |PARTITIONED|
+                                          {
+                                            -- AGGREGATE  |LOCAL|
+                                              -- STREAM_SELECT  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                          }
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$55(ASC), $$61(ASC)]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$55]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$80(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- RTREE_SEARCH  |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  |PARTITIONED|
+                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..18c2244
--- /dev/null
+++ 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
@@ -0,0 +1,92 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |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[$$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|
+                                  -- STREAM_SELECT  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- BTREE_SEARCH  |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  |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  |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|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- BTREE_SEARCH  |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  |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  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..ccf2c9c
--- /dev/null
+++ 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
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$47(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |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[$$38]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..ccf2c9c
--- /dev/null
+++ 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
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$47(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$38]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- STREAM_SELECT  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |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[$$38]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$38(ASC), $$39(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                        -- STREAM_SELECT  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1ad81d2
--- /dev/null
+++ 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
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$52(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|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |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|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1ad81d2
--- /dev/null
+++ 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
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$52(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|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |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|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[04/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ForwardOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ForwardOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ForwardOperator.java
new file mode 100644
index 0000000..db11712
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/ForwardOperator.java
@@ -0,0 +1,111 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT 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 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.LogicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.expressions.IVariableTypeEnvironment;
+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;
+import org.apache.hyracks.algebricks.core.algebra.typing.ITypingContext;
+import org.apache.hyracks.algebricks.core.algebra.typing.OpRefTypeEnvPointer;
+import org.apache.hyracks.algebricks.core.algebra.typing.PropagatingTypeEnvironment;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalExpressionReferenceTransform;
+import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
+
+/**
+ * Forward operator is used to forward data to different NCs based on a range map that is computed dynamically
+ * by doing a pass over the data itself to infer the range map. The operator takes two inputs:
+ * 1. Tuples/data (at index 0). The data is forwarded to the range-based connector which routes it to the target NC.
+ * 2. Range map (at index 1). The range map will be stored in Hyracks context, and the connector will pick it up.
+ * Forward operator will receive the range map when it is broadcast by the operator generating the range map after which
+ * the forward operator will start forwarding the data.
+ */
+public class ForwardOperator extends AbstractLogicalOperator {
+
+    private final String rangeMapKey;
+    private final Mutable<ILogicalExpression> rangeMapExpression;
+
+    public ForwardOperator(String rangeMapKey, Mutable<ILogicalExpression> rangeMapExpression) {
+        super();
+        this.rangeMapKey = rangeMapKey;
+        this.rangeMapExpression = rangeMapExpression;
+    }
+
+    public String getRangeMapKey() {
+        return rangeMapKey;
+    }
+
+    public Mutable<ILogicalExpression> getRangeMapExpression() {
+        return rangeMapExpression;
+    }
+
+    @Override
+    public LogicalOperatorTag getOperatorTag() {
+        return LogicalOperatorTag.FORWARD;
+    }
+
+    @Override
+    public void recomputeSchema() throws AlgebricksException {
+        // schema is equal to the schema of the data source at idx 0
+        setSchema(inputs.get(0).getValue().getSchema());
+    }
+
+    @Override
+    public boolean acceptExpressionTransform(ILogicalExpressionReferenceTransform visitor) throws AlgebricksException {
+        return visitor.transform(rangeMapExpression);
+    }
+
+    @Override
+    public <R, T> R accept(ILogicalOperatorVisitor<R, T> visitor, T arg) throws AlgebricksException {
+        return visitor.visitForwardOperator(this, arg);
+    }
+
+    @Override
+    public boolean isMap() {
+        return false;
+    }
+
+    @Override
+    public VariablePropagationPolicy getVariablePropagationPolicy() {
+        return new VariablePropagationPolicy() {
+
+            @Override
+            public void propagateVariables(IOperatorSchema target, IOperatorSchema... sources)
+                    throws AlgebricksException {
+                // propagate the variables of the data source at idx 0
+                if (sources.length > 0) {
+                    target.addAllVariables(sources[0]);
+                }
+            }
+        };
+    }
+
+    @Override
+    public IVariableTypeEnvironment computeOutputTypeEnvironment(ITypingContext ctx) throws AlgebricksException {
+        // propagate the type environment of the data source at idx 0
+        ITypeEnvPointer[] envPointers = new ITypeEnvPointer[] { new OpRefTypeEnvPointer(inputs.get(0), ctx) };
+        return new PropagatingTypeEnvironment(ctx.getExpressionTypeComputer(), ctx.getMissableTypeComputer(),
+                ctx.getMetadataProvider(), TypePropagationPolicy.ALL, envPointers);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
index d0aea60..9d853eb 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/CardinalityInferenceVisitor.java
@@ -31,11 +31,12 @@ import org.apache.hyracks.algebricks.core.algebra.expressions.ConstantExpression
 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.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -276,6 +277,11 @@ public class CardinalityInferenceVisitor implements ILogicalOperatorVisitor<Long
     }
 
     @Override
+    public Long visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return op.getInputs().get(0).getValue().accept(this, arg);
+    }
+
+    @Override
     public Long visitIntersectOperator(IntersectOperator op, Void arg) throws AlgebricksException {
         long cardinality = UNKNOWN;
         for (Mutable<ILogicalOperator> inputOpRef : op.getInputs()) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
index d0d121f..16fc1ed 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/FDsAndEquivClassesVisitor.java
@@ -53,6 +53,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -560,6 +561,12 @@ public class FDsAndEquivClassesVisitor implements ILogicalOperatorVisitor<Void,
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, IOptimizationContext ctx) throws AlgebricksException {
+        propagateFDsAndEquivClasses(op, ctx);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, IOptimizationContext ctx) throws AlgebricksException {
         setEmptyFDsEqClasses(op, ctx);
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
index 90c0067..2b5e569 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismOperatorVisitor.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -595,6 +596,18 @@ public class IsomorphismOperatorVisitor implements ILogicalOperatorVisitor<Boole
     }
 
     @Override
+    public Boolean visitForwardOperator(ForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        AbstractLogicalOperator argOperator = (AbstractLogicalOperator) arg;
+        if (argOperator.getOperatorTag() != LogicalOperatorTag.FORWARD) {
+            return Boolean.FALSE;
+        }
+        ForwardOperator otherOp = (ForwardOperator) copyAndSubstituteVar(op, arg);
+        ILogicalExpression rangeMapExp = op.getRangeMapExpression().getValue();
+        ILogicalExpression otherRangeMapExp = otherOp.getRangeMapExpression().getValue();
+        return rangeMapExp.equals(otherRangeMapExp) && op.getRangeMapKey().equals(otherOp.getRangeMapKey());
+    }
+
+    @Override
     public Boolean visitSinkOperator(SinkOperator op, ILogicalOperator arg) throws AlgebricksException {
         return true;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
index 2caa252..742d485 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/IsomorphismVariableMappingVisitor.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -284,6 +285,12 @@ public class IsomorphismVariableMappingVisitor implements ILogicalOperatorVisito
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        mapVariablesStandard(op, arg);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, ILogicalOperator arg) throws AlgebricksException {
         mapVariablesStandard(op, arg);
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
index e0210cc..0196db6 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalOperatorDeepCopyWithNewVariablesVisitor.java
@@ -43,6 +43,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -587,6 +588,14 @@ public class LogicalOperatorDeepCopyWithNewVariablesVisitor
     }
 
     @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, ILogicalOperator arg) throws AlgebricksException {
+        ForwardOperator opCopy = new ForwardOperator(op.getRangeMapKey(),
+                exprDeepCopyVisitor.deepCopyExpressionReference(op.getRangeMapExpression()));
+        deepCopyInputsAnnotationsAndExecutionMode(op, arg, opCopy);
+        return opCopy;
+    }
+
+    @Override
     public ILogicalOperator visitDelegateOperator(DelegateOperator op, ILogicalOperator arg)
             throws AlgebricksException {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
index 6dfe254..7d3d676 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/LogicalPropertiesVisitor.java
@@ -30,11 +30,12 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractAssi
 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.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -289,6 +290,12 @@ public class LogicalPropertiesVisitor implements ILogicalOperatorVisitor<Void, I
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, IOptimizationContext arg) throws AlgebricksException {
+        // TODO Auto-generated method stub
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, IOptimizationContext arg) throws AlgebricksException {
         // TODO Auto-generated method stub
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
index 0db0f74..c6f0c14 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/OperatorDeepCopyVisitor.java
@@ -40,6 +40,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -342,6 +343,11 @@ public class OperatorDeepCopyVisitor implements ILogicalOperatorVisitor<ILogical
     }
 
     @Override
+    public ILogicalOperator visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return new ForwardOperator(op.getRangeMapKey(), deepCopyExpressionRef(op.getRangeMapExpression()));
+    }
+
+    @Override
     public ILogicalOperator visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
         return new SinkOperator();
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
index c96276f..f36f604 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/PrimaryKeyVariablesVisitor.java
@@ -32,11 +32,12 @@ import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 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.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -278,6 +279,11 @@ public class PrimaryKeyVariablesVisitor implements ILogicalOperatorVisitor<Void,
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, IOptimizationContext arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, IOptimizationContext arg)
             throws AlgebricksException {
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
index ec96d48..5d0ef6a 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/ProducedVariableVisitor.java
@@ -36,11 +36,12 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnne
 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.DelegateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.DelegateOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -278,6 +279,11 @@ public class ProducedVariableVisitor implements ILogicalOperatorVisitor<Void, Vo
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
index 59ccd84..70ccf6d 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SchemaVariableVisitor.java
@@ -39,6 +39,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -318,6 +319,13 @@ public class SchemaVariableVisitor implements ILogicalOperatorVisitor<Void, Void
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        // only consider variables from the branch of the data source
+        VariableUtilities.getLiveVariables(op.getInputs().get(0).getValue(), schemaVariables);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Void arg) throws AlgebricksException {
         standardLayout(op);
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
index 3587e29..c62f555 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/SubstituteVariableVisitor.java
@@ -39,6 +39,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -460,6 +461,14 @@ public class SubstituteVariableVisitor
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Pair<LogicalVariable, LogicalVariable> arg)
+            throws AlgebricksException {
+        op.getRangeMapExpression().getValue().substituteVar(arg.first, arg.second);
+        substVarTypes(op, arg);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Pair<LogicalVariable, LogicalVariable> pair)
             throws AlgebricksException {
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
index e66809e..2c68697 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/logical/visitors/UsedVariableVisitor.java
@@ -41,6 +41,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -138,50 +139,43 @@ public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void>
             switch (physOp.getOperatorTag()) {
                 case BROADCAST_EXCHANGE:
                 case ONE_TO_ONE_EXCHANGE:
-                case RANDOM_MERGE_EXCHANGE: {
+                case RANDOM_MERGE_EXCHANGE:
+                case SEQUENTIAL_MERGE_EXCHANGE:
                     // No variables used.
                     break;
-                }
-                case HASH_PARTITION_EXCHANGE: {
-                    HashPartitionExchangePOperator concreteOp = (HashPartitionExchangePOperator) physOp;
-                    usedVariables.addAll(concreteOp.getHashFields());
+                case HASH_PARTITION_EXCHANGE:
+                    HashPartitionExchangePOperator hashPartitionPOp = (HashPartitionExchangePOperator) physOp;
+                    usedVariables.addAll(hashPartitionPOp.getHashFields());
                     break;
-                }
-                case HASH_PARTITION_MERGE_EXCHANGE: {
-                    HashPartitionMergeExchangePOperator concreteOp = (HashPartitionMergeExchangePOperator) physOp;
-                    usedVariables.addAll(concreteOp.getPartitionFields());
-                    for (OrderColumn orderCol : concreteOp.getOrderColumns()) {
+                case HASH_PARTITION_MERGE_EXCHANGE:
+                    HashPartitionMergeExchangePOperator hashMergePOp = (HashPartitionMergeExchangePOperator) physOp;
+                    usedVariables.addAll(hashMergePOp.getPartitionFields());
+                    for (OrderColumn orderCol : hashMergePOp.getOrderColumns()) {
                         usedVariables.add(orderCol.getColumn());
                     }
                     break;
-                }
-                case SORT_MERGE_EXCHANGE: {
-                    SortMergeExchangePOperator concreteOp = (SortMergeExchangePOperator) physOp;
-                    for (OrderColumn orderCol : concreteOp.getSortColumns()) {
+                case SORT_MERGE_EXCHANGE:
+                    SortMergeExchangePOperator sortMergePOp = (SortMergeExchangePOperator) physOp;
+                    for (OrderColumn orderCol : sortMergePOp.getSortColumns()) {
                         usedVariables.add(orderCol.getColumn());
                     }
                     break;
-                }
-                case RANGE_PARTITION_EXCHANGE: {
-                    RangePartitionExchangePOperator concreteOp = (RangePartitionExchangePOperator) physOp;
-                    for (OrderColumn partCol : concreteOp.getPartitioningFields()) {
+                case RANGE_PARTITION_EXCHANGE:
+                    RangePartitionExchangePOperator rangePartitionPOp = (RangePartitionExchangePOperator) physOp;
+                    for (OrderColumn partCol : rangePartitionPOp.getPartitioningFields()) {
                         usedVariables.add(partCol.getColumn());
                     }
                     break;
-                }
-                case RANGE_PARTITION_MERGE_EXCHANGE: {
-                    RangePartitionMergeExchangePOperator concreteOp = (RangePartitionMergeExchangePOperator) physOp;
-                    for (OrderColumn partCol : concreteOp.getPartitioningFields()) {
+                case RANGE_PARTITION_MERGE_EXCHANGE:
+                    RangePartitionMergeExchangePOperator rangeMergePOp = (RangePartitionMergeExchangePOperator) physOp;
+                    for (OrderColumn partCol : rangeMergePOp.getPartitioningFields()) {
                         usedVariables.add(partCol.getColumn());
                     }
                     break;
-                }
-                case RANDOM_PARTITION_EXCHANGE: {
+                case RANDOM_PARTITION_EXCHANGE:
                     break;
-                }
-                default: {
+                default:
                     throw new AlgebricksException("Unhandled physical operator tag '" + physOp.getOperatorTag() + "'.");
-                }
             }
         }
         return null;
@@ -439,6 +433,12 @@ public class UsedVariableVisitor implements ILogicalOperatorVisitor<Void, Void>
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Void arg) throws AlgebricksException {
+        op.getRangeMapExpression().getValue().getUsedVariables(usedVariables);
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Void arg) {
         return null;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
index 78e96a4..0c08369 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractStableSortPOperator.java
@@ -19,6 +19,7 @@
 package org.apache.hyracks.algebricks.core.algebra.operators.physical;
 
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 
@@ -29,16 +30,20 @@ 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.base.OperatorAnnotations;
+import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
 import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.LocalOrderProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.OrderColumn;
+import org.apache.hyracks.algebricks.core.algebra.properties.OrderedPartitionedProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 
@@ -67,16 +72,26 @@ public abstract class AbstractStableSortPOperator extends AbstractPhysicalOperat
     }
 
     @Override
-    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
-            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
-        AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
-        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator sortOp,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext ctx) {
+        if (sortOp.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
             if (orderProp == null) {
-                computeLocalProperties(op);
+                computeLocalProperties(sortOp);
             }
-            StructuralPropertiesVector[] r = new StructuralPropertiesVector[] { new StructuralPropertiesVector(
-                    IPartitioningProperty.UNPARTITIONED, Collections.singletonList(orderProp)) };
-            return new PhysicalRequirements(r, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+            StructuralPropertiesVector[] requiredProp = new StructuralPropertiesVector[1];
+            IPartitioningProperty partitioning;
+            INodeDomain targetNodeDomain = ctx.getComputationNodeDomain();
+            if (isFullParallel((AbstractLogicalOperator) sortOp, targetNodeDomain, ctx)) {
+                // partitioning requirement: input data is re-partitioned on sort columns (global ordering)
+                // TODO(ali): static range map implementation should be fixed to require ORDERED_PARTITION and come here
+                partitioning = new OrderedPartitionedProperty(Arrays.asList(sortColumns), targetNodeDomain);
+            } else {
+                // partitioning requirement: input data is unpartitioned (i.e. must be merged at one site)
+                partitioning = IPartitioningProperty.UNPARTITIONED;
+            }
+            // local requirement: each partition must be locally ordered
+            requiredProp[0] = new StructuralPropertiesVector(partitioning, Collections.singletonList(orderProp));
+            return new PhysicalRequirements(requiredProp, IPartitioningRequirementsCoordinator.NO_COORDINATION);
         } else {
             return emptyUnaryRequirements();
         }
@@ -123,4 +138,27 @@ public abstract class AbstractStableSortPOperator extends AbstractPhysicalOperat
     public boolean expensiveThanMaterialization() {
         return true;
     }
+
+    /**
+     * When true, the sort operator requires ORDERED_PARTITION (only applicable to dynamic version for now).
+     * Conditions:
+     * 1. Execution mode == partitioned
+     * 2. Dynamic range map was not disabled by some checks
+     * 3. User didn't disable it
+     * 4. User didn't provide static range map
+     * 5. Physical sort operator is not in-memory
+     * 6. There are at least two partitions in the cluster
+     * @param sortOp the sort operator
+     * @param clusterDomain the partitions specification of the cluster
+     * @param ctx optimization context
+     * @return true if the sort operator should be full parallel sort, false otherwise.
+     */
+    private boolean isFullParallel(AbstractLogicalOperator sortOp, INodeDomain clusterDomain,
+            IOptimizationContext ctx) {
+        return sortOp.getAnnotations().get(OperatorAnnotations.USE_DYNAMIC_RANGE) != Boolean.FALSE
+                && !sortOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)
+                && sortOp.getPhysicalOperator().getOperatorTag() == PhysicalOperatorTag.STABLE_SORT
+                && clusterDomain.cardinality() != null && clusterDomain.cardinality() > 1
+                && ctx.getPhysicalOptimizationConfig().getSortParallel();
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ForwardPOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ForwardPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ForwardPOperator.java
new file mode 100644
index 0000000..11c584e
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/ForwardPOperator.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.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+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.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.BroadcastPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningRequirementsCoordinator;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
+import org.apache.hyracks.api.dataflow.value.RecordDescriptor;
+import org.apache.hyracks.dataflow.std.misc.ForwardOperatorDescriptor;
+
+/**
+ * <pre>
+ * {@see {@link ForwardOperator} and {@link ForwardOperatorDescriptor}}
+ * idx0: Input data source --
+ *                           |-- forward op.
+ * idx1: RangeMap generator--
+ * </pre>
+ */
+public class ForwardPOperator extends AbstractPhysicalOperator {
+
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.FORWARD;
+    }
+
+    /**
+     * Forward operator requires that the global aggregate operator broadcasts the range map. No required properties at
+     * the data source input.
+     * @param op {@see {@link ForwardOperator}}
+     * @param requiredByParent parent's requirements, which are not enforced for now, as we only explore one plan
+     * @param context the optimization context
+     * @return broadcast requirement at input 1; empty requirements at input 0; No coordination between the two.
+     */
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector requiredByParent, IOptimizationContext context) {
+        // broadcast the range map to the cluster node domain
+        INodeDomain targetDomain = context.getComputationNodeDomain();
+        List<ILocalStructuralProperty> noProp = new ArrayList<>();
+        StructuralPropertiesVector[] requiredAtInputs = new StructuralPropertiesVector[2];
+        requiredAtInputs[0] = StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
+        requiredAtInputs[1] = new StructuralPropertiesVector(new BroadcastPartitioningProperty(targetDomain), noProp);
+        return new PhysicalRequirements(requiredAtInputs, IPartitioningRequirementsCoordinator.NO_COORDINATION);
+    }
+
+    /**
+     * Forward operator delivers whatever properties delivered by the input located at index = 0 (tuples source op).
+     * Subtree at index 0 must compute its delivered properties before any call to this method
+     * @param op forward logical operator
+     * @param context {@link IOptimizationContext}
+     * @throws AlgebricksException
+     */
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        ILogicalOperator dataSourceOperator = op.getInputs().get(0).getValue();
+        deliveredProperties = dataSourceOperator.getDeliveredPhysicalProperties().clone();
+    }
+
+    /**
+     * The output record descriptor of forward operator is the same as the output record descriptor of the data source
+     * which is located at index 0.
+     * @param builder Hyracks job builder
+     * @param context job generation context
+     * @param op {@see {@link ForwardOperator}}
+     * @param propagatedSchema not used
+     * @param inputSchemas schemas of all inputs
+     * @param outerPlanSchema not used
+     * @throws AlgebricksException
+     */
+    @Override
+    public void contributeRuntimeOperator(IHyracksJobBuilder builder, JobGenContext context, ILogicalOperator op,
+            IOperatorSchema propagatedSchema, IOperatorSchema[] inputSchemas, IOperatorSchema outerPlanSchema)
+            throws AlgebricksException {
+        ForwardOperator forwardOp = (ForwardOperator) op;
+        RecordDescriptor dataInputDescriptor = JobGenHelper.mkRecordDescriptor(
+                context.getTypeEnvironment(forwardOp.getInputs().get(0).getValue()), inputSchemas[0], context);
+        ForwardOperatorDescriptor forwardDescriptor =
+                new ForwardOperatorDescriptor(builder.getJobSpec(), forwardOp.getRangeMapKey(), dataInputDescriptor);
+        builder.contributeHyracksOperator(forwardOp, forwardDescriptor);
+        ILogicalOperator dataSource = forwardOp.getInputs().get(0).getValue();
+        builder.contributeGraphEdge(dataSource, 0, forwardOp, 0);
+        ILogicalOperator rangemapSource = forwardOp.getInputs().get(1).getValue();
+        builder.contributeGraphEdge(rangemapSource, 0, forwardOp, 1);
+    }
+
+    @Override
+    public Pair<int[], int[]> getInputOutputDependencyLabels(ILogicalOperator op) {
+        int[] outputDependencyLabels = new int[] { 1 };
+        int[] inputDependencyLabels = new int[] { 1, 0 };
+        return new Pair<>(inputDependencyLabels, outputDependencyLabels);
+    }
+
+    @Override
+    public boolean isMicroOperator() {
+        return false;
+    }
+
+    @Override
+    public boolean expensiveThanMaterialization() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
index 6630d32..aeb9ac7 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionExchangePOperator.java
@@ -42,27 +42,40 @@ import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirement
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.data.IBinaryComparatorFactoryProvider;
-import org.apache.hyracks.algebricks.data.INormalizedKeyComputerFactoryProvider;
 import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
 import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
-import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
-import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.common.data.partition.range.DynamicFieldRangePartitionComputerFactory;
 import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.StaticFieldRangePartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningConnectorDescriptor;
 
 public class RangePartitionExchangePOperator extends AbstractExchangePOperator {
 
     private List<OrderColumn> partitioningFields;
     private INodeDomain domain;
-    private IRangeMap rangeMap;
+    private RangeMap rangeMap;
+    private final boolean rangeMapIsComputedAtRunTime;
+    private final String rangeMapKeyInContext;
 
-    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
-            IRangeMap rangeMap) {
+    private RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain, RangeMap rangeMap,
+            boolean rangeMapIsComputedAtRunTime, String rangeMapKeyInContext) {
         this.partitioningFields = partitioningFields;
         this.domain = domain;
         this.rangeMap = rangeMap;
+        this.rangeMapIsComputedAtRunTime = rangeMapIsComputedAtRunTime;
+        this.rangeMapKeyInContext = rangeMapKeyInContext;
+    }
+
+    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, String rangeMapKeyInContext,
+            INodeDomain domain) {
+        this(partitioningFields, domain, null, true, rangeMapKeyInContext);
+    }
+
+    public RangePartitionExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
+            RangeMap rangeMap) {
+        this(partitioningFields, domain, rangeMap, false, "");
     }
 
     @Override
@@ -80,8 +93,7 @@ public class RangePartitionExchangePOperator extends AbstractExchangePOperator {
 
     @Override
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
-        IPartitioningProperty p =
-                new OrderedPartitionedProperty(new ArrayList<OrderColumn>(partitioningFields), domain);
+        IPartitioningProperty p = new OrderedPartitionedProperty(new ArrayList<>(partitioningFields), domain);
         this.deliveredProperties = new StructuralPropertiesVector(p, new LinkedList<ILocalStructuralProperty>());
     }
 
@@ -97,32 +109,31 @@ public class RangePartitionExchangePOperator extends AbstractExchangePOperator {
         int n = partitioningFields.size();
         int[] sortFields = new int[n];
         IBinaryComparatorFactory[] comps = new IBinaryComparatorFactory[n];
-
-        INormalizedKeyComputerFactoryProvider nkcfProvider = context.getNormalizedKeyComputerFactoryProvider();
-        INormalizedKeyComputerFactory nkcf = null;
-
         IVariableTypeEnvironment env = context.getTypeEnvironment(op);
         int i = 0;
         for (OrderColumn oc : partitioningFields) {
             LogicalVariable var = oc.getColumn();
             sortFields[i] = opSchema.findVariable(var);
             Object type = env.getVarType(var);
-            OrderKind order = oc.getOrder();
-            if (i == 0 && nkcfProvider != null && type != null) {
-                nkcf = nkcfProvider.getNormalizedKeyComputerFactory(type, order == OrderKind.ASC);
-            }
             IBinaryComparatorFactoryProvider bcfp = context.getBinaryComparatorFactoryProvider();
             comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
             i++;
         }
-        ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
-        IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, tpcf);
-        return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
+        FieldRangePartitionComputerFactory partitionerFactory;
+        if (rangeMapIsComputedAtRunTime) {
+            partitionerFactory = new DynamicFieldRangePartitionComputerFactory(sortFields, comps, rangeMapKeyInContext,
+                    op.getSourceLocation());
+        } else {
+            partitionerFactory = new StaticFieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
+        }
+
+        IConnectorDescriptor conn = new MToNPartitioningConnectorDescriptor(spec, partitionerFactory);
+        return new Pair<>(conn, null);
     }
 
     @Override
     public String toString() {
-        return getOperatorTag().toString() + " " + partitioningFields + " SPLIT COUNT:" + rangeMap.getSplitCount();
+        final String splitCount = rangeMap == null ? "" : " SPLIT COUNT:" + Integer.toString(rangeMap.getSplitCount());
+        return getOperatorTag().toString() + " " + partitioningFields + splitCount;
     }
-
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
index ec32a53..b015193 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/RangePartitionMergeExchangePOperator.java
@@ -53,18 +53,18 @@ import org.apache.hyracks.api.dataflow.value.IBinaryComparatorFactory;
 import org.apache.hyracks.api.dataflow.value.INormalizedKeyComputerFactory;
 import org.apache.hyracks.api.dataflow.value.ITuplePartitionComputerFactory;
 import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
-import org.apache.hyracks.dataflow.common.data.partition.range.FieldRangePartitionComputerFactory;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.StaticFieldRangePartitionComputerFactory;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
 
 public class RangePartitionMergeExchangePOperator extends AbstractExchangePOperator {
 
     private List<OrderColumn> partitioningFields;
     private INodeDomain domain;
-    private IRangeMap rangeMap;
+    private RangeMap rangeMap;
 
     public RangePartitionMergeExchangePOperator(List<OrderColumn> partitioningFields, INodeDomain domain,
-            IRangeMap rangeMap) {
+            RangeMap rangeMap) {
         this.partitioningFields = partitioningFields;
         this.domain = domain;
         this.rangeMap = rangeMap;
@@ -143,7 +143,7 @@ public class RangePartitionMergeExchangePOperator extends AbstractExchangePOpera
             comps[i] = bcfp.getBinaryComparatorFactory(type, oc.getOrder() == OrderKind.ASC);
             i++;
         }
-        ITuplePartitionComputerFactory tpcf = new FieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
+        ITuplePartitionComputerFactory tpcf = new StaticFieldRangePartitionComputerFactory(sortFields, comps, rangeMap);
         IConnectorDescriptor conn = new MToNPartitioningMergingConnectorDescriptor(spec, tpcf, sortFields, comps, nkcf);
         return new Pair<IConnectorDescriptor, TargetConstraint>(conn, null);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SequentialMergeExchangePOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SequentialMergeExchangePOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SequentialMergeExchangePOperator.java
new file mode 100644
index 0000000..df0b446
--- /dev/null
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/SequentialMergeExchangePOperator.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.hyracks.algebricks.core.algebra.operators.physical;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.algebricks.common.utils.Pair;
+import org.apache.hyracks.algebricks.core.algebra.base.IHyracksJobBuilder;
+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.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.IOperatorSchema;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.properties.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
+import org.apache.hyracks.api.dataflow.IConnectorDescriptor;
+import org.apache.hyracks.api.job.IConnectorDescriptorRegistry;
+import org.apache.hyracks.dataflow.std.connectors.MToOneSequentialMergingConnectorDescriptor;
+
+public class SequentialMergeExchangePOperator extends AbstractExchangePOperator {
+    @Override
+    public PhysicalOperatorTag getOperatorTag() {
+        return PhysicalOperatorTag.SEQUENTIAL_MERGE_EXCHANGE;
+    }
+
+    @Override
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
+            IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
+        return emptyUnaryRequirements();
+    }
+
+    @Override
+    public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context)
+            throws AlgebricksException {
+        AbstractLogicalOperator childOp = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
+        List<ILocalStructuralProperty> childLocalProps = childOp.getDeliveredPhysicalProperties().getLocalProperties();
+        List<ILocalStructuralProperty> localProperties;
+        if (childLocalProps != null) {
+            localProperties = new ArrayList<>(childLocalProps);
+        } else {
+            localProperties = new ArrayList<>(0);
+        }
+
+        deliveredProperties = new StructuralPropertiesVector(IPartitioningProperty.UNPARTITIONED, localProperties);
+    }
+
+    @Override
+    public Pair<IConnectorDescriptor, IHyracksJobBuilder.TargetConstraint> createConnectorDescriptor(
+            IConnectorDescriptorRegistry spec, ILogicalOperator op, IOperatorSchema opSchema, JobGenContext context)
+            throws AlgebricksException {
+        IConnectorDescriptor connector = new MToOneSequentialMergingConnectorDescriptor(spec);
+        return new Pair<>(connector, IHyracksJobBuilder.TargetConstraint.ONE);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitor.java
----------------------------------------------------------------------
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 99ed738..77f052e 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
@@ -40,6 +40,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -465,6 +466,13 @@ public class LogicalOperatorPrettyPrintVisitor extends AbstractLogicalOperatorPr
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Integer indent) throws AlgebricksException {
+        addIndent(indent)
+                .append("forward: range-map = " + op.getRangeMapExpression().getValue().accept(exprVisitor, indent));
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("sink");
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/prettyprint/LogicalOperatorPrettyPrintVisitorJson.java
----------------------------------------------------------------------
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 f1f1f3b..4a17cc6 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
@@ -44,6 +44,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -647,6 +648,14 @@ public class LogicalOperatorPrettyPrintVisitorJson extends AbstractLogicalOperat
     }
 
     @Override
+    public Void visitForwardOperator(ForwardOperator op, Integer indent) throws AlgebricksException {
+        addIndent(indent).append("\"operator\": \"forward\"");
+        addIndent(indent).append("\"expressions\": \""
+                + op.getRangeMapExpression().getValue().accept(exprVisitor, indent).replace('"', ' ') + "\"");
+        return null;
+    }
+
+    @Override
     public Void visitSinkOperator(SinkOperator op, Integer indent) throws AlgebricksException {
         addIndent(indent).append("\"operator\": \"sink\"");
         return null;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
index e7e98a5..b006a1e 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/LocalOrderProperty.java
@@ -121,20 +121,7 @@ public final class LocalOrderProperty implements ILocalStructuralProperty {
         Iterator<OrderColumn> currentColumnIterator = orderColumns.iterator();
 
         // Returns true if requiredColumnIterator is a prefix of currentColumnIterator.
-        return isPrefixOf(requiredColumnIterator, currentColumnIterator);
-    }
-
-    private <T> boolean isPrefixOf(Iterator<T> requiredColumnIterator, Iterator<T> currentColumnIterator) {
-        while (requiredColumnIterator.hasNext()) {
-            T oc = requiredColumnIterator.next();
-            if (!currentColumnIterator.hasNext()) {
-                return false;
-            }
-            if (!oc.equals(currentColumnIterator.next())) {
-                return false;
-            }
-        }
-        return true;
+        return PropertiesUtil.isPrefixOf(requiredColumnIterator, currentColumnIterator);
     }
 
     // Gets normalized  ordering columns, where each column variable is a representative variable of its equivalence

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
index f2fed13..1c00e45 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/PropertiesUtil.java
@@ -200,7 +200,7 @@ public class PropertiesUtil {
      * @param target
      * @return true iff pref is a prefix of target
      */
-    private static <T> boolean isPrefixOf(Iterator<T> pref, Iterator<T> target) {
+    public static <T> boolean isPrefixOf(Iterator<T> pref, Iterator<T> target) {
         while (pref.hasNext()) {
             T v = pref.next();
             if (!target.hasNext()) {
@@ -213,50 +213,65 @@ public class PropertiesUtil {
         return true;
     }
 
+    /**
+     * Normalizes or reduces the order columns argument based on the functional dependencies argument. The caller is
+     * responsible for taking caution as to how to handle the returned object since this method either returns the same
+     * object that is passed or returns a new object.
+     * @param orderColumns the order columns that are to be normalized
+     * @param functionalDependencies {@link FunctionalDependency}
+     * @return a new normalized object if normalization is applied. Otherwise, the same argument object is returned.
+     */
     public static List<OrderColumn> applyFDsToOrderColumns(List<OrderColumn> orderColumns,
-            List<FunctionalDependency> fds) {
-        // the set of vars. is ordered
-        // so we try the variables in order from last to first
-        if (fds == null || fds.isEmpty()) {
+            List<FunctionalDependency> functionalDependencies) {
+        if (functionalDependencies == null || functionalDependencies.isEmpty()) {
             return orderColumns;
         }
 
+        // the set of vars. is ordered
+        // so we try the variables in order from last to first
         int deleted = 0;
+        boolean[] removedColumns = new boolean[orderColumns.size()];
         for (int i = orderColumns.size() - 1; i >= 0; i--) {
-            for (FunctionalDependency fdep : fds) {
-                if (impliedByPrefix(orderColumns, i, fdep)) {
-                    orderColumns.set(i, null);
+            for (FunctionalDependency functionalDependency : functionalDependencies) {
+                if (impliedByPrefix(orderColumns, i, functionalDependency)) {
+                    removedColumns[i] = true;
                     deleted++;
                     break;
                 }
             }
         }
-        List<OrderColumn> norm = new ArrayList<>(orderColumns.size() - deleted);
-        for (OrderColumn oc : orderColumns) {
-            if (oc != null) {
-                norm.add(oc);
+        List<OrderColumn> normalizedColumns = new ArrayList<>(orderColumns.size() - deleted);
+        for (int i = 0; i < orderColumns.size(); i++) {
+            if (!removedColumns[i]) {
+                normalizedColumns.add(orderColumns.get(i));
             }
         }
-        return norm;
+
+        return normalizedColumns;
     }
 
+    /**
+     * Normalizes or reduces the order columns argument based on the equivalenceClasses argument. The caller is
+     * responsible for taking caution as to how to handle the returned object since this method either returns the same
+     * object that is passed or returns a new object.
+     * @param orderColumns the order columns that are to be normalized
+     * @param equivalenceClasses {@link EquivalenceClass}
+     * @return a new normalized object if normalization is applied. Otherwise, the same argument object is returned.
+     */
     public static List<OrderColumn> replaceOrderColumnsByEqClasses(List<OrderColumn> orderColumns,
             Map<LogicalVariable, EquivalenceClass> equivalenceClasses) {
         if (equivalenceClasses == null || equivalenceClasses.isEmpty()) {
             return orderColumns;
         }
         List<OrderColumn> norm = new ArrayList<>();
-        for (OrderColumn v : orderColumns) {
-            EquivalenceClass ec = equivalenceClasses.get(v.getColumn());
-            if (ec == null) {
-                norm.add(v);
-            } else {
-                if (ec.representativeIsConst()) {
-                    // trivially satisfied, so the var. can be removed
-                } else {
-                    norm.add(new OrderColumn(ec.getVariableRepresentative(), v.getOrder()));
-                }
+        for (OrderColumn orderColumn : orderColumns) {
+            EquivalenceClass columnEQClass = equivalenceClasses.get(orderColumn.getColumn());
+            if (columnEQClass == null) {
+                norm.add(orderColumn);
+            } else if (!columnEQClass.representativeIsConst()) {
+                norm.add(new OrderColumn(columnEQClass.getVariableRepresentative(), orderColumn.getOrder()));
             }
+            // else columnEQClass rep. is constant, i.e. trivially satisfied, so the var. can be removed
         }
         return norm;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
index deb98b0..548a29f 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/visitors/ILogicalOperatorVisitor.java
@@ -27,6 +27,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -124,4 +125,5 @@ public interface ILogicalOperatorVisitor<R, T> {
 
     public R visitTokenizeOperator(TokenizeOperator op, T arg) throws AlgebricksException;
 
+    public R visitForwardOperator(ForwardOperator op, T arg) throws AlgebricksException;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/config/AlgebricksConfig.java
----------------------------------------------------------------------
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 8779777..15bb54b 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
@@ -25,4 +25,6 @@ public class AlgebricksConfig {
 
     public static final String ALGEBRICKS_LOGGER_NAME = "org.apache.hyracks.algebricks";
     public static final Logger ALGEBRICKS_LOGGER = LogManager.getLogger(ALGEBRICKS_LOGGER_NAME);
+    public static final int SORT_SAMPLES = 100;
+    public static final boolean SORT_PARALLEL = true;
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java
index 6fa378b..e3b1868 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/jobgen/impl/ConnectorPolicyAssignmentPolicy.java
@@ -24,6 +24,7 @@ import org.apache.hyracks.api.dataflow.connectors.IConnectorPolicyAssignmentPoli
 import org.apache.hyracks.api.dataflow.connectors.PipeliningConnectorPolicy;
 import org.apache.hyracks.api.dataflow.connectors.SendSideMaterializedPipeliningConnectorPolicy;
 import org.apache.hyracks.dataflow.std.connectors.MToNPartitioningMergingConnectorDescriptor;
+import org.apache.hyracks.dataflow.std.connectors.MToOneSequentialMergingConnectorDescriptor;
 
 public class ConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignmentPolicy {
     private static final long serialVersionUID = 1L;
@@ -33,7 +34,8 @@ public class ConnectorPolicyAssignmentPolicy implements IConnectorPolicyAssignme
     @Override
     public IConnectorPolicy getConnectorPolicyAssignment(IConnectorDescriptor c, int nProducers, int nConsumers,
             int[] fanouts) {
-        if (c instanceof MToNPartitioningMergingConnectorDescriptor) {
+        if (c instanceof MToNPartitioningMergingConnectorDescriptor
+                || c instanceof MToOneSequentialMergingConnectorDescriptor) {
             return senderSideMaterializePolicy;
         } else {
             return pipeliningPolicy;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
index 8eb9b90..a2a0ca1 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/HeuristicOptimizer.java
@@ -37,29 +37,20 @@ import org.apache.logging.log4j.Level;
 
 public class HeuristicOptimizer {
 
-    public static PhysicalOperatorTag[] hyracksOperators =
-            new PhysicalOperatorTag[] { PhysicalOperatorTag.DATASOURCE_SCAN, PhysicalOperatorTag.BTREE_SEARCH,
-                    PhysicalOperatorTag.EXTERNAL_GROUP_BY, PhysicalOperatorTag.HASH_GROUP_BY,
-                    PhysicalOperatorTag.HDFS_READER, PhysicalOperatorTag.HYBRID_HASH_JOIN,
-                    PhysicalOperatorTag.IN_MEMORY_HASH_JOIN, PhysicalOperatorTag.NESTED_LOOP,
-                    PhysicalOperatorTag.PRE_SORTED_DISTINCT_BY, PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY,
-                    PhysicalOperatorTag.REPLICATE, PhysicalOperatorTag.STABLE_SORT, PhysicalOperatorTag.UNION_ALL };
-    public static PhysicalOperatorTag[] hyraxOperatorsBelowWhichJobGenIsDisabled = new PhysicalOperatorTag[] {};
-
-    public static boolean isHyracksOp(PhysicalOperatorTag opTag) {
-        for (PhysicalOperatorTag t : hyracksOperators) {
-            if (t == opTag) {
-                return true;
-            }
-        }
-        return false;
-    }
-
     private final IOptimizationContext context;
     private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites;
     private final List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites;
     private final ILogicalPlan plan;
 
+    private static final PhysicalOperatorTag[] hyracksOperators = new PhysicalOperatorTag[] {
+            PhysicalOperatorTag.DATASOURCE_SCAN, PhysicalOperatorTag.BTREE_SEARCH,
+            PhysicalOperatorTag.EXTERNAL_GROUP_BY, PhysicalOperatorTag.HASH_GROUP_BY, PhysicalOperatorTag.HDFS_READER,
+            PhysicalOperatorTag.HYBRID_HASH_JOIN, PhysicalOperatorTag.IN_MEMORY_HASH_JOIN,
+            PhysicalOperatorTag.NESTED_LOOP, PhysicalOperatorTag.PRE_SORTED_DISTINCT_BY,
+            PhysicalOperatorTag.PRE_CLUSTERED_GROUP_BY, PhysicalOperatorTag.REPLICATE, PhysicalOperatorTag.STABLE_SORT,
+            PhysicalOperatorTag.UNION_ALL, PhysicalOperatorTag.FORWARD };
+    public static final PhysicalOperatorTag[] hyraxOperatorsBelowWhichJobGenIsDisabled = new PhysicalOperatorTag[] {};
+
     public HeuristicOptimizer(ILogicalPlan plan,
             List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> logicalRewrites,
             List<Pair<AbstractRuleController, List<IAlgebraicRewriteRule>>> physicalRewrites,
@@ -70,6 +61,15 @@ public class HeuristicOptimizer {
         this.physicalRewrites = physicalRewrites;
     }
 
+    public static boolean isHyracksOp(PhysicalOperatorTag opTag) {
+        for (PhysicalOperatorTag t : hyracksOperators) {
+            if (t == opTag) {
+                return true;
+            }
+        }
+        return false;
+    }
+
     public void optimize() throws AlgebricksException {
         if (plan == null) {
             return;
@@ -129,7 +129,6 @@ public class HeuristicOptimizer {
         if (AlgebricksConfig.ALGEBRICKS_LOGGER.isTraceEnabled()) {
             AlgebricksConfig.ALGEBRICKS_LOGGER.trace("Starting physical optimizations.\n");
         }
-        // PhysicalOptimizationsUtil.computeFDsAndEquivalenceClasses(plan);
         runOptimizationSets(plan, physicalRewrites);
     }
 


[16/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance_ps.plan
new file mode 100644
index 0000000..07563e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
new file mode 100644
index 0000000..e061424
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-fuzzyeq-edit-distance_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$14(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$14(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
new file mode 100644
index 0000000..29b0fef
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |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_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let_ps.plan
new file mode 100644
index 0000000..b9bd479
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-complex/olist-edit-distance-check-let_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |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_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$27(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f145eac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.plan
@@ -0,0 +1,165 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$63(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$69][$$53]  |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  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$53]  |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  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |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[$$53]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$69][$$53]  |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  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$53]  |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  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..560158f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/leftouterjoin-probe-pidx-with-join-jaccard-check-idx_01_ps.plan
@@ -0,0 +1,353 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$63(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |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|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- BTREE_SEARCH  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$54][$$idRight_0]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                              -- EXTERNAL_GROUP_BY[$$135, $$133]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- HASH_PARTITION_EXCHANGE [$$135, $$133]  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- STREAM_SELECT  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                              -- UNNEST  |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), $$i(ASC)]  |PARTITIONED|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- UNNEST  |PARTITIONED|
+                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- REPLICATE  |PARTITIONED|
+                                                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                              -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                    -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                                {
+                                                                                                                  -- AGGREGATE  |LOCAL|
+                                                                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                }
+                                                                                                          -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                            -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                    {
+                                                                                                                      -- AGGREGATE  |LOCAL|
+                                                                                                                        -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                    }
+                                                                                                              -- 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  |PARTITIONED|
+                                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- UNNEST  |PARTITIONED|
+                                                                  -- ASSIGN  |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), $$i(ASC)]  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                                        -- UNNEST  |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|
+                                                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- ASSIGN  |PARTITIONED|
+                                                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                      -- REPLICATE  |PARTITIONED|
+                                                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                          -- ASSIGN  |PARTITIONED|
+                                                                                            -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                  -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                      -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                              {
+                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                              }
+                                                                                                        -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                          -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                  {
+                                                                                                                    -- AGGREGATE  |LOCAL|
+                                                                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                  }
+                                                                                                            -- 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  |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[$$53]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |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|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$54][$$idRight_0]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$idRight_0]  |PARTITIONED|
+                                                        -- EXTERNAL_GROUP_BY[$$135, $$133]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- HASH_PARTITION_EXCHANGE [$$135, $$133]  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- STREAM_SELECT  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$prefixTokenRight][$$prefixTokenLeft]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$prefixTokenRight]  |PARTITIONED|
+                                                                        -- UNNEST  |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), $$i(ASC)]  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- UNNEST  |PARTITIONED|
+                                                                                                -- ASSIGN  |PARTITIONED|
+                                                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                  -- REPLICATE  |PARTITIONED|
+                                                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                        -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                            -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                              -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                  -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                                          {
+                                                                                                                            -- AGGREGATE  |LOCAL|
+                                                                                                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                          }
+                                                                                                                    -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                                      -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                              {
+                                                                                                                                -- AGGREGATE  |LOCAL|
+                                                                                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                              }
+                                                                                                                        -- 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  |PARTITIONED|
+                                                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$prefixTokenLeft]  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- UNNEST  |PARTITIONED|
+                                                                            -- ASSIGN  |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), $$i(ASC)]  |PARTITIONED|
+                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                            -- HYBRID_HASH_JOIN [$$token][$$tokenGroupped]  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                  -- UNNEST  |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|
+                                                                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                      -- ASSIGN  |PARTITIONED|
+                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- REPLICATE  |PARTITIONED|
+                                                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                                                    -- ASSIGN  |PARTITIONED|
+                                                                                                      -- RUNNING_AGGREGATE  |PARTITIONED|
+                                                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                                                          -- SORT_MERGE_EXCHANGE [$$140(ASC), $$tokenGroupped(ASC) ]  |PARTITIONED|
+                                                                                                            -- STABLE_SORT [$$140(ASC), $$tokenGroupped(ASC)]  |PARTITIONED|
+                                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                -- EXTERNAL_GROUP_BY[$$147]  |PARTITIONED|
+                                                                                                                        {
+                                                                                                                          -- AGGREGATE  |LOCAL|
+                                                                                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                        }
+                                                                                                                  -- HASH_PARTITION_EXCHANGE [$$147]  |PARTITIONED|
+                                                                                                                    -- EXTERNAL_GROUP_BY[$$token]  |PARTITIONED|
+                                                                                                                            {
+                                                                                                                              -- AGGREGATE  |LOCAL|
+                                                                                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                                                                            }
+                                                                                                                      -- 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  |PARTITIONED|
+                                                                                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-join/ngram-contains_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |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|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..73103df
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-core_ps.plan
@@ -0,0 +1,53 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$36(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$36(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 [$$38][$$39]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |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 [$$38][$$39]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$38]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$39]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..08b3940
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/loj-sugar_ps.plan
@@ -0,0 +1,53 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$29(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$29(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 [$$31][$$32]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                              -- ASSIGN  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |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 [$$31][$$32]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[03/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/rewriter/base/PhysicalOptimizationConfig.java
----------------------------------------------------------------------
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 07ff0ab..d879d36 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
@@ -20,6 +20,8 @@ package org.apache.hyracks.algebricks.core.rewriter.base;
 
 import java.util.Properties;
 
+import org.apache.hyracks.algebricks.core.config.AlgebricksConfig;
+
 public class PhysicalOptimizationConfig {
     private static final int MB = 1048576;
 
@@ -31,10 +33,11 @@ public class PhysicalOptimizationConfig {
     private static final String MAX_FRAMES_FOR_TEXTSEARCH = "MAX_FRAMES_FOR_TEXTSEARCH";
     private static final String FUDGE_FACTOR = "FUDGE_FACTOR";
     private static final String MAX_RECORDS_PER_FRAME = "MAX_RECORDS_PER_FRAME";
-
     private static final String DEFAULT_HASH_GROUP_TABLE_SIZE = "DEFAULT_HASH_GROUP_TABLE_SIZE";
     private static final String DEFAULT_EXTERNAL_GROUP_TABLE_SIZE = "DEFAULT_EXTERNAL_GROUP_TABLE_SIZE";
     private static final String DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE = "DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE";
+    private static final String SORT_PARALLEL = "SORT_PARALLEL";
+    private static final String SORT_SAMPLES = "SORT_SAMPLES";
 
     private Properties properties = new Properties();
 
@@ -143,6 +146,22 @@ public class PhysicalOptimizationConfig {
         setInt(DEFAULT_IN_MEM_HASH_JOIN_TABLE_SIZE, tableSize);
     }
 
+    public boolean getSortParallel() {
+        return getBoolean(SORT_PARALLEL, AlgebricksConfig.SORT_PARALLEL);
+    }
+
+    public void setSortParallel(boolean sortParallel) {
+        setBoolean(SORT_PARALLEL, sortParallel);
+    }
+
+    public int getSortSamples() {
+        return getInt(SORT_SAMPLES, AlgebricksConfig.SORT_SAMPLES);
+    }
+
+    public void setSortSamples(int sortSamples) {
+        setInt(SORT_SAMPLES, sortSamples);
+    }
+
     private void setInt(String property, int value) {
         properties.setProperty(property, Integer.toString(value));
     }
@@ -167,4 +186,16 @@ public class PhysicalOptimizationConfig {
             return Double.parseDouble(value);
     }
 
+    private void setBoolean(String property, boolean value) {
+        properties.setProperty(property, Boolean.toString(value));
+    }
+
+    private boolean getBoolean(String property, boolean defaultValue) {
+        String value = properties.getProperty(property);
+        if (value == null) {
+            return defaultValue;
+        } else {
+            return Boolean.parseBoolean(value);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java
index 8ada0ac..d6895e3 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/DotFormatGenerator.java
@@ -29,12 +29,16 @@ import org.apache.commons.lang3.tuple.Pair;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalPlan;
-import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 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.AbstractOperatorWithNestedPlans;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.api.constraints.Constraint;
+import org.apache.hyracks.api.constraints.expressions.ConstraintExpression;
+import org.apache.hyracks.api.constraints.expressions.LValueConstraintExpression;
+import org.apache.hyracks.api.constraints.expressions.PartitionCountExpression;
+import org.apache.hyracks.api.constraints.expressions.PartitionLocationExpression;
 import org.apache.hyracks.api.dataflow.ActivityId;
 import org.apache.hyracks.api.dataflow.ConnectorDescriptorId;
 import org.apache.hyracks.api.dataflow.IActivity;
@@ -45,15 +49,13 @@ import org.apache.hyracks.api.job.JobSpecification;
 
 public class DotFormatGenerator {
 
-    private DotFormatGenerator() {
-    }
+    private final LogicalOperatorDotVisitor dotVisitor = new LogicalOperatorDotVisitor();
 
     /**
-     * Generates DOT format for {@link JobActivityGraph} that can be visualized
-     * using any DOT format visualizer.
+     * Generates DOT format plan for {@link JobActivityGraph} that can be visualized using any DOT format visualizer.
      *
      * @param jobActivityGraph The job activity graph
-     * @return DOT format
+     * @return DOT format plan
      */
     public static String generate(final JobActivityGraph jobActivityGraph) {
         final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("JobActivityGraph"));
@@ -146,92 +148,74 @@ public class DotFormatGenerator {
     }
 
     /**
-     * Generates DOT format for {@link JobSpecification} that can be visualized
-     * using any DOT format visualizer.
+     * Generates DOT format plan for {@link JobSpecification} that can be visualized using any DOT format visualizer.
      *
      * @param jobSpecification The job specification
-     * @return DOT format
+     * @return DOT format plan
      */
     public static String generate(final JobSpecification jobSpecification) {
         final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("JobSpecification"));
         final Map<ConnectorDescriptorId, IConnectorDescriptor> connectorMap = jobSpecification.getConnectorMap();
-        final Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> cOp =
+        final Set<Constraint> constraints = jobSpecification.getUserConstraints();
+        Map<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> cOp =
                 jobSpecification.getConnectorOperatorMap();
-        ConnectorDescriptorId connectorId;
-        IConnectorDescriptor connector;
-        IOperatorDescriptor leftOperator;
-        IOperatorDescriptor rightOperator;
-        DotFormatBuilder.Node sourceNode;
-        DotFormatBuilder.Node destinationNode;
-        String source;
-        String destination;
-        String edgeLabel;
-        for (Map.Entry<ConnectorDescriptorId, Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>>> entry : cOp
-                .entrySet()) {
-            connectorId = entry.getKey();
-            connector = connectorMap.get(connectorId);
-            edgeLabel = connector.getClass().getName().substring(connector.getClass().getName().lastIndexOf(".") + 1);
-            edgeLabel += "-" + connectorId;
-            leftOperator = entry.getValue().getLeft().getLeft();
-            rightOperator = entry.getValue().getRight().getLeft();
-            source = leftOperator.getClass().getName()
-                    .substring(leftOperator.getClass().getName().lastIndexOf(".") + 1);
-            sourceNode =
-                    graphBuilder.createNode(DotFormatBuilder.StringValue.of(leftOperator.getOperatorId().toString()),
-                            DotFormatBuilder.StringValue.of(leftOperator.toString() + "-" + source));
-            destination = rightOperator.getClass().getName()
-                    .substring(rightOperator.getClass().getName().lastIndexOf(".") + 1);
-            destinationNode =
-                    graphBuilder.createNode(DotFormatBuilder.StringValue.of(rightOperator.getOperatorId().toString()),
-                            DotFormatBuilder.StringValue.of(rightOperator.toString() + "-" + destination));
-            graphBuilder.createEdge(sourceNode, destinationNode).setLabel(DotFormatBuilder.StringValue.of(edgeLabel));
-        }
-
+        cOp.forEach((connId, srcAndDest) -> addToGraph(graphBuilder, constraints, connectorMap, connId, srcAndDest));
         return graphBuilder.getDotDocument();
     }
 
     /**
-     * Generates DOT format for {@link ILogicalPlan} that can be visualized
-     * using any DOT format visualizer.
+     * Generates DOT format plan for {@link ILogicalPlan} that can be visualized using any DOT format visualizer.
      *
      * @param plan  The logical plan
-     * @param dotVisitor    The DOT visitor
-     * @return DOT format
-     * @throws AlgebricksException
+     * @param showDetails whether to show the details of the operator like physical properties
+     * @return DOT format plan
+     * @throws AlgebricksException When one operator throws an exception while visiting it.
      */
-    public static String generate(ILogicalPlan plan, LogicalOperatorDotVisitor dotVisitor) throws AlgebricksException {
-        final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("Plan"));
+    public String generate(ILogicalPlan plan, boolean showDetails) throws AlgebricksException {
         ILogicalOperator root = plan.getRoots().get(0).getValue();
-        generateNode(graphBuilder, root, dotVisitor, new HashSet<>());
+        return generate(root, showDetails);
+    }
+
+    /**
+     * Generates DOT format plan considering "startingOp" as the root operator.
+     *
+     * @param startingOp the starting operator
+     * @param showDetails whether to show the details of the operator like physical properties
+     * @return DOT format plan
+     * @throws AlgebricksException When one operator throws an exception while visiting it.
+     */
+    public String generate(ILogicalOperator startingOp, boolean showDetails) throws AlgebricksException {
+        final DotFormatBuilder graphBuilder = new DotFormatBuilder(DotFormatBuilder.StringValue.of("Plan"));
+        generateNode(graphBuilder, startingOp, showDetails, new HashSet<>());
         return graphBuilder.getDotDocument();
     }
 
-    public static void generateNode(DotFormatBuilder dotBuilder, ILogicalOperator op,
-            LogicalOperatorDotVisitor dotVisitor, Set<ILogicalOperator> operatorsVisited) throws AlgebricksException {
-        DotFormatBuilder.StringValue destinationNodeLabel = formatStringOf(op, dotVisitor);
+    private void generateNode(DotFormatBuilder dotBuilder, ILogicalOperator op, boolean showDetails,
+            Set<ILogicalOperator> operatorsVisited) throws AlgebricksException {
+        DotFormatBuilder.StringValue destinationNodeLabel = formatStringOf(op, showDetails);
         DotFormatBuilder.Node destinationNode = dotBuilder
                 .createNode(DotFormatBuilder.StringValue.of(Integer.toString(op.hashCode())), destinationNodeLabel);
         DotFormatBuilder.StringValue sourceNodeLabel;
         DotFormatBuilder.Node sourceNode;
         for (Mutable<ILogicalOperator> child : op.getInputs()) {
-            sourceNodeLabel = formatStringOf(child.getValue(), dotVisitor);
+            sourceNodeLabel = formatStringOf(child.getValue(), showDetails);
             sourceNode = dotBuilder.createNode(
                     DotFormatBuilder.StringValue.of(Integer.toString(child.getValue().hashCode())), sourceNodeLabel);
             dotBuilder.createEdge(sourceNode, destinationNode);
             if (!operatorsVisited.contains(child.getValue())) {
-                generateNode(dotBuilder, child.getValue(), dotVisitor, operatorsVisited);
+                generateNode(dotBuilder, child.getValue(), showDetails, operatorsVisited);
             }
         }
         if (((AbstractLogicalOperator) op).hasNestedPlans()) {
             ILogicalOperator nestedOperator;
             for (ILogicalPlan nestedPlan : ((AbstractOperatorWithNestedPlans) op).getNestedPlans()) {
                 nestedOperator = nestedPlan.getRoots().get(0).getValue();
-                sourceNodeLabel = formatStringOf(nestedOperator, dotVisitor);
+                sourceNodeLabel = formatStringOf(nestedOperator, showDetails);
                 sourceNode = dotBuilder.createNode(
                         DotFormatBuilder.StringValue.of(Integer.toString(nestedOperator.hashCode())), sourceNodeLabel);
                 dotBuilder.createEdge(sourceNode, destinationNode).setLabel(DotFormatBuilder.StringValue.of("subplan"));
                 if (!operatorsVisited.contains(nestedOperator)) {
-                    generateNode(dotBuilder, nestedOperator, dotVisitor, operatorsVisited);
+                    generateNode(dotBuilder, nestedOperator, showDetails, operatorsVisited);
                 }
             }
         }
@@ -246,7 +230,7 @@ public class DotFormatGenerator {
             sourceNode = destinationNode;
             for (int i = 0; i < replicateOperator.getOutputs().size(); i++) {
                 replicateOutput = replicateOperator.getOutputs().get(i).getValue();
-                destinationNodeLabel = formatStringOf(replicateOutput, dotVisitor);
+                destinationNodeLabel = formatStringOf(replicateOutput, showDetails);
                 destinationNode = dotBuilder.createNode(
                         DotFormatBuilder.StringValue.of(Integer.toString(replicateOutput.hashCode())),
                         destinationNodeLabel);
@@ -261,16 +245,52 @@ public class DotFormatGenerator {
         operatorsVisited.add(op);
     }
 
-    private static DotFormatBuilder.StringValue formatStringOf(ILogicalOperator operator,
-            LogicalOperatorDotVisitor dotVisitor) throws AlgebricksException {
-        String formattedString = operator.accept(dotVisitor, null).trim();
-        IPhysicalOperator physicalOperator = ((AbstractLogicalOperator) operator).getPhysicalOperator();
-        if (physicalOperator != null) {
-            formattedString += "\\n" + physicalOperator.toString().trim() + " |" + operator.getExecutionMode() + "|";
-        } else {
-            formattedString += "\\n|" + operator.getExecutionMode() + "|";
-        }
-
+    private DotFormatBuilder.StringValue formatStringOf(ILogicalOperator operator, boolean showDetails)
+            throws AlgebricksException {
+        String formattedString = operator.accept(dotVisitor, showDetails).trim();
         return DotFormatBuilder.StringValue.of(formattedString);
     }
+
+    private static void addToGraph(DotFormatBuilder graph, Set<Constraint> constraints,
+            Map<ConnectorDescriptorId, IConnectorDescriptor> connMap, ConnectorDescriptorId connId,
+            Pair<Pair<IOperatorDescriptor, Integer>, Pair<IOperatorDescriptor, Integer>> srcAndDest) {
+        IConnectorDescriptor connector = connMap.get(connId);
+        String edgeLabel;
+        edgeLabel = connector.getClass().getName().substring(connector.getClass().getName().lastIndexOf(".") + 1);
+        edgeLabel += "-" + connId;
+        IOperatorDescriptor sourceOp = srcAndDest.getLeft().getLeft();
+        IOperatorDescriptor destOp = srcAndDest.getRight().getLeft();
+        StringBuilder source = new StringBuilder(
+                sourceOp.getClass().getName().substring(sourceOp.getClass().getName().lastIndexOf(".") + 1));
+        StringBuilder destination = new StringBuilder(
+                destOp.getClass().getName().substring(destOp.getClass().getName().lastIndexOf(".") + 1));
+        // constraints
+        for (Constraint constraint : constraints) {
+            LValueConstraintExpression lvalue = constraint.getLValue();
+            if (lvalue.getTag() == ConstraintExpression.ExpressionTag.PARTITION_COUNT) {
+                PartitionCountExpression count = (PartitionCountExpression) lvalue;
+                if (count.getOperatorDescriptorId().equals(sourceOp.getOperatorId())) {
+                    source.append("\n").append(constraint);
+                }
+                if (count.getOperatorDescriptorId().equals(destOp.getOperatorId())) {
+                    destination.append("\n").append(constraint);
+                }
+            } else if (lvalue.getTag() == ConstraintExpression.ExpressionTag.PARTITION_LOCATION) {
+                PartitionLocationExpression location = (PartitionLocationExpression) lvalue;
+                if (location.getOperatorDescriptorId().equals(sourceOp.getOperatorId())) {
+                    source.append("\n").append(constraint);
+                }
+                if (location.getOperatorDescriptorId().equals(destOp.getOperatorId())) {
+                    destination.append("\n").append(constraint);
+                }
+            }
+        }
+        DotFormatBuilder.Node sourceNode =
+                graph.createNode(DotFormatBuilder.StringValue.of(sourceOp.getOperatorId().toString()),
+                        DotFormatBuilder.StringValue.of(sourceOp.toString() + "-" + source));
+        DotFormatBuilder.Node destinationNode =
+                graph.createNode(DotFormatBuilder.StringValue.of(destOp.getOperatorId().toString()),
+                        DotFormatBuilder.StringValue.of(destOp.toString() + "-" + destination));
+        graph.createEdge(sourceNode, destinationNode).setLabel(DotFormatBuilder.StringValue.of(edgeLabel));
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/utils/LogicalOperatorDotVisitor.java
----------------------------------------------------------------------
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 2cb2d35..113d205 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
@@ -18,14 +18,20 @@
  */
 package org.apache.hyracks.algebricks.core.utils;
 
+import static org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType.LOCAL_GROUPING_PROPERTY;
+import static org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty.PropertyType.LOCAL_ORDER_PROPERTY;
+
 import java.util.List;
+import java.util.Map;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
 import org.apache.hyracks.algebricks.common.utils.Triple;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.base.IPhysicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractUnnestMapOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperator;
@@ -35,6 +41,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOper
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistributeResultOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.EmptyTupleSourceOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IndexInsertDeleteUpsertOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
@@ -62,9 +69,14 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestMapOpe
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.UnnestOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.WriteResultOperator;
+import org.apache.hyracks.algebricks.core.algebra.properties.DefaultNodeGroupDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.ILocalStructuralProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.INodeDomain;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProperty;
+import org.apache.hyracks.algebricks.core.algebra.properties.IPhysicalPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.visitors.ILogicalOperatorVisitor;
 
-public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String, Void> {
+public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String, Boolean> {
 
     private final StringBuilder stringBuilder;
 
@@ -82,161 +94,214 @@ public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String
     }
 
     @Override
-    public String visitAggregateOperator(AggregateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitAggregateOperator(AggregateOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("aggregate ").append(str(op.getVariables())).append(" <- ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitRunningAggregateOperator(RunningAggregateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitRunningAggregateOperator(RunningAggregateOperator op, Boolean showDetails)
+            throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("running-aggregate ").append(str(op.getVariables())).append(" <- ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Void noArgs) throws AlgebricksException {
+    public String visitEmptyTupleSourceOperator(EmptyTupleSourceOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("empty-tuple-source");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitGroupByOperator(GroupByOperator op, Void noArgs) throws AlgebricksException {
+    public String visitGroupByOperator(GroupByOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("group by").append(op.isGroupAll() ? " (all)" : "").append(" (");
-        pprintVeList(op.getGroupByList());
+        printVariableAndExprList(op.getGroupByList());
         stringBuilder.append(") decor (");
-        pprintVeList(op.getDecorList());
+        printVariableAndExprList(op.getDecorList());
         stringBuilder.append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitDistinctOperator(DistinctOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDistinctOperator(DistinctOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("distinct (");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
         stringBuilder.append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitInnerJoinOperator(InnerJoinOperator op, Void noArgs) throws AlgebricksException {
+    public String visitInnerJoinOperator(InnerJoinOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("join (").append(op.getCondition().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Void noArgs) throws AlgebricksException {
+    public String visitLeftOuterJoinOperator(LeftOuterJoinOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("left outer join (").append(op.getCondition().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Void noArgs) throws AlgebricksException {
+    public String visitNestedTupleSourceOperator(NestedTupleSourceOperator op, Boolean showDetails)
+            throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("nested tuple source");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitOrderOperator(OrderOperator op, Void noArgs) throws AlgebricksException {
+    public String visitOrderOperator(OrderOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("order ");
         for (Pair<OrderOperator.IOrder, Mutable<ILogicalExpression>> p : op.getOrderExpressions()) {
             if (op.getTopK() != -1) {
                 stringBuilder.append("(topK: ").append(op.getTopK()).append(") ");
             }
-            String fst = getOrderString(p.first);
-            stringBuilder.append("(").append(fst).append(", ").append(p.second.getValue().toString()).append(") ");
+            stringBuilder.append("(");
+            switch (p.first.getKind()) {
+                case ASC:
+                    stringBuilder.append("ASC");
+                    break;
+                case DESC:
+                    stringBuilder.append("DESC");
+                    break;
+                default:
+                    final Mutable<ILogicalExpression> expressionRef = p.first.getExpressionRef();
+                    stringBuilder.append(expressionRef == null ? "null" : expressionRef.toString());
+            }
+            stringBuilder.append(", ").append(p.second.getValue().toString()).append(") ");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
-    private String getOrderString(OrderOperator.IOrder first) {
-        switch (first.getKind()) {
-            case ASC:
-                return "ASC";
-            case DESC:
-                return "DESC";
-            default:
-                return first.getExpressionRef().toString();
-        }
-    }
-
     @Override
-    public String visitAssignOperator(AssignOperator op, Void noArgs) throws AlgebricksException {
+    public String visitAssignOperator(AssignOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("assign ").append(str(op.getVariables())).append(" <- ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitWriteOperator(WriteOperator op, Void noArgs) throws AlgebricksException {
+    public String visitWriteOperator(WriteOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("write ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitDistributeResultOperator(DistributeResultOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDistributeResultOperator(DistributeResultOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("distribute result ");
-        pprintExprList(op.getExpressions());
+        printExprList(op.getExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitWriteResultOperator(WriteResultOperator op, Void noArgs) throws AlgebricksException {
+    public String visitWriteResultOperator(WriteResultOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("load ").append(str(op.getDataSource())).append(" from ")
                 .append(op.getPayloadExpression().getValue().toString()).append(" partitioned by ");
-        pprintExprList(op.getKeyExpressions());
+        printExprList(op.getKeyExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSelectOperator(SelectOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSelectOperator(SelectOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("select (").append(op.getCondition().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitProjectOperator(ProjectOperator op, Void noArgs) throws AlgebricksException {
+    public String visitProjectOperator(ProjectOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("project ").append("(").append(op.getVariables()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSubplanOperator(SubplanOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSubplanOperator(SubplanOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("subplan {}");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitUnionOperator(UnionAllOperator op, Void noArgs) throws AlgebricksException {
+    public String visitUnionOperator(UnionAllOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("union");
         for (Triple<LogicalVariable, LogicalVariable, LogicalVariable> v : op.getVariableMappings()) {
             stringBuilder.append(" (").append(v.first).append(", ").append(v.second).append(", ").append(v.third)
                     .append(")");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitIntersectOperator(IntersectOperator op, Void noArgs) throws AlgebricksException {
+    public String visitIntersectOperator(IntersectOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("intersect (");
         stringBuilder.append('[');
@@ -261,154 +326,183 @@ public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String
             stringBuilder.append(']');
         }
         stringBuilder.append("])");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitUnnestOperator(UnnestOperator op, Void noArgs) throws AlgebricksException {
+    public String visitUnnestOperator(UnnestOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("unnest ").append(op.getVariable());
         if (op.getPositionalVariable() != null) {
             stringBuilder.append(" at ").append(op.getPositionalVariable());
         }
         stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Void noArgs) throws AlgebricksException {
+    public String visitLeftOuterUnnestOperator(LeftOuterUnnestOperator op, Boolean showDetails)
+            throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("outer-unnest ").append(op.getVariable());
         if (op.getPositionalVariable() != null) {
             stringBuilder.append(" at ").append(op.getPositionalVariable());
         }
         stringBuilder.append(" <- ").append(op.getExpressionRef().getValue().toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitUnnestMapOperator(UnnestMapOperator op, Void noArgs) throws AlgebricksException {
+    public String visitUnnestMapOperator(UnnestMapOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
-        printAbstractUnnestMapOperator(op, "unnest-map");
-        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
-        appendLimitInformation(stringBuilder, op.getOutputLimit());
+        printAbstractUnnestMapOperator(op, "unnest-map", showDetails);
+        appendSelectConditionInformation(op.getSelectCondition());
+        appendLimitInformation(op.getOutputLimit());
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Void noArgs)
+    public String visitLeftOuterUnnestMapOperator(LeftOuterUnnestMapOperator op, Boolean showDetails)
             throws AlgebricksException {
         stringBuilder.setLength(0);
-        printAbstractUnnestMapOperator(op, "left-outer-unnest-map");
+        printAbstractUnnestMapOperator(op, "left-outer-unnest-map", showDetails);
         return stringBuilder.toString();
     }
 
-    private void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature) {
+    private void printAbstractUnnestMapOperator(AbstractUnnestMapOperator op, String opSignature, boolean show) {
         stringBuilder.append(opSignature).append(" ").append(op.getVariables()).append(" <- ")
                 .append(op.getExpressionRef().getValue().toString());
-        appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
+        appendFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSchema(op, show);
+        appendAnnotations(op, show);
+        appendPhysicalOperatorInfo(op, show);
     }
 
     @Override
-    public String visitDataScanOperator(DataSourceScanOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDataScanOperator(DataSourceScanOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("data-scan ").append(op.getProjectVariables()).append("<-").append(op.getVariables())
                 .append(" <- ").append(op.getDataSource());
-        appendFilterInformation(stringBuilder, op.getMinFilterVars(), op.getMaxFilterVars());
-        appendSelectConditionInformation(stringBuilder, op.getSelectCondition());
-        appendLimitInformation(stringBuilder, op.getOutputLimit());
+        appendFilterInformation(op.getMinFilterVars(), op.getMaxFilterVars());
+        appendSelectConditionInformation(op.getSelectCondition());
+        appendLimitInformation(op.getOutputLimit());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
-    private void appendFilterInformation(StringBuilder plan, List<LogicalVariable> minFilterVars,
-            List<LogicalVariable> maxFilterVars) {
+    private void appendFilterInformation(List<LogicalVariable> minFilterVars, List<LogicalVariable> maxFilterVars) {
         if (minFilterVars != null || maxFilterVars != null) {
-            plan.append(" with filter on");
+            stringBuilder.append(" with filter on");
         }
         if (minFilterVars != null) {
-            plan.append(" min:").append(minFilterVars);
+            stringBuilder.append(" min:").append(minFilterVars);
         }
         if (maxFilterVars != null) {
-            plan.append(" max:").append(maxFilterVars);
+            stringBuilder.append(" max:").append(maxFilterVars);
         }
     }
 
-    private Void appendSelectConditionInformation(StringBuilder plan, Mutable<ILogicalExpression> condition)
-            throws AlgebricksException {
+    private void appendSelectConditionInformation(Mutable<ILogicalExpression> condition) throws AlgebricksException {
         if (condition != null) {
-            plan.append(" condition:").append(condition.getValue().toString());
+            stringBuilder.append(" condition:").append(condition.getValue().toString());
         }
-        return null;
     }
 
-    private Void appendLimitInformation(StringBuilder plan, long outputLimit) throws AlgebricksException {
+    private void appendLimitInformation(long outputLimit) throws AlgebricksException {
         if (outputLimit >= 0) {
-            plan.append(" limit:").append(String.valueOf(outputLimit));
+            stringBuilder.append(" limit:").append(String.valueOf(outputLimit));
         }
-        return null;
     }
 
     @Override
-    public String visitLimitOperator(LimitOperator op, Void noArgs) throws AlgebricksException {
+    public String visitLimitOperator(LimitOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("limit ").append(op.getMaxObjects().getValue().toString());
         ILogicalExpression offset = op.getOffset().getValue();
         if (offset != null) {
             stringBuilder.append(", ").append(offset.toString());
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitExchangeOperator(ExchangeOperator op, Void noArgs) throws AlgebricksException {
+    public String visitExchangeOperator(ExchangeOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("exchange");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitScriptOperator(ScriptOperator op, Void noArgs) throws AlgebricksException {
+    public String visitScriptOperator(ScriptOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("script (in: ").append(op.getInputVariables()).append(") (out: ")
                 .append(op.getOutputVariables()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitReplicateOperator(ReplicateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitReplicateOperator(ReplicateOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("replicate");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSplitOperator(SplitOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSplitOperator(SplitOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         Mutable<ILogicalExpression> branchingExpression = op.getBranchingExpression();
         stringBuilder.append("split ").append(branchingExpression.getValue().toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitMaterializeOperator(MaterializeOperator op, Void noArgs) throws AlgebricksException {
+    public String visitMaterializeOperator(MaterializeOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("materialize");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Void noArgs)
-            throws AlgebricksException {
+    public String visitInsertDeleteUpsertOperator(InsertDeleteUpsertOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         String header = getIndexOpString(op.getOperation());
         stringBuilder.append(header).append(str(op.getDataSource())).append(" from record: ")
                 .append(op.getPayloadExpression().getValue().toString());
         if (op.getAdditionalNonFilteringExpressions() != null) {
             stringBuilder.append(", meta: ");
-            pprintExprList(op.getAdditionalNonFilteringExpressions());
+            printExprList(op.getAdditionalNonFilteringExpressions());
         }
         stringBuilder.append(" partitioned by ");
-        pprintExprList(op.getPrimaryKeyExpressions());
+        printExprList(op.getPrimaryKeyExpressions());
         if (op.getOperation() == Kind.UPSERT) {
             stringBuilder.append(" out: ([record-before-upsert:").append(op.getBeforeOpRecordVar());
             if (op.getBeforeOpAdditionalNonFilteringVars() != null) {
@@ -419,27 +513,32 @@ public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String
         if (op.isBulkload()) {
             stringBuilder.append(" [bulkload]");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Void noArgs)
-            throws AlgebricksException {
+    public String visitIndexInsertDeleteUpsertOperator(IndexInsertDeleteUpsertOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         String header = getIndexOpString(op.getOperation());
         stringBuilder.append(header).append(op.getIndexName()).append(" on ")
                 .append(str(op.getDataSourceIndex().getDataSource())).append(" from ");
         if (op.getOperation() == Kind.UPSERT) {
             stringBuilder.append(" replace:");
-            pprintExprList(op.getPrevSecondaryKeyExprs());
+            printExprList(op.getPrevSecondaryKeyExprs());
             stringBuilder.append(" with:");
-            pprintExprList(op.getSecondaryKeyExpressions());
+            printExprList(op.getSecondaryKeyExpressions());
         } else {
-            pprintExprList(op.getSecondaryKeyExpressions());
+            printExprList(op.getSecondaryKeyExpressions());
         }
         if (op.isBulkload()) {
             stringBuilder.append(" [bulkload]");
         }
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
@@ -452,60 +551,143 @@ public class LogicalOperatorDotVisitor implements ILogicalOperatorVisitor<String
             case UPSERT:
                 return "upsert into ";
         }
-        return null;
+        return "";
     }
 
     @Override
-    public String visitTokenizeOperator(TokenizeOperator op, Void noArgs) throws AlgebricksException {
+    public String visitTokenizeOperator(TokenizeOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append("tokenize ").append(str(op.getTokenizeVars())).append(" <- ");
-        pprintExprList(op.getSecondaryKeyExpressions());
+        printExprList(op.getSecondaryKeyExpressions());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitSinkOperator(SinkOperator op, Void noArgs) throws AlgebricksException {
+    public String visitSinkOperator(SinkOperator op, Boolean showDetails) {
         stringBuilder.setLength(0);
         stringBuilder.append("sink");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
     @Override
-    public String visitDelegateOperator(DelegateOperator op, Void noArgs) throws AlgebricksException {
+    public String visitDelegateOperator(DelegateOperator op, Boolean showDetails) throws AlgebricksException {
         stringBuilder.setLength(0);
         stringBuilder.append(op.toString());
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
+        return stringBuilder.toString();
+    }
+
+    @Override
+    public String visitForwardOperator(ForwardOperator op, Boolean showDetails) throws AlgebricksException {
+        stringBuilder.setLength(0);
+        stringBuilder.append("forward(").append(op.getRangeMapExpression().getValue().toString()).append(")");
+        appendSchema(op, showDetails);
+        appendAnnotations(op, showDetails);
+        appendPhysicalOperatorInfo(op, showDetails);
         return stringBuilder.toString();
     }
 
-    private void pprintExprList(List<Mutable<ILogicalExpression>> expressions) {
+    private void printExprList(List<Mutable<ILogicalExpression>> expressions) {
+        stringBuilder.append("[");
+        expressions.forEach(exprRef -> stringBuilder.append(exprRef.getValue().toString()).append(", "));
+        stringBuilder.append("]");
+    }
+
+    private void printVariableAndExprList(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> variableExprList) {
         stringBuilder.append("[");
         boolean first = true;
-        for (Mutable<ILogicalExpression> exprRef : expressions) {
+        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> variableExpressionPair : variableExprList) {
             if (first) {
                 first = false;
             } else {
-                stringBuilder.append(", ");
+                stringBuilder.append("; ");
+            }
+            if (variableExpressionPair.first != null) {
+                stringBuilder.append(variableExpressionPair.first).append(" := ").append(variableExpressionPair.second);
+            } else {
+                stringBuilder.append(variableExpressionPair.second.getValue().toString());
             }
-            stringBuilder.append(exprRef.getValue().toString());
         }
         stringBuilder.append("]");
     }
 
-    private void pprintVeList(List<Pair<LogicalVariable, Mutable<ILogicalExpression>>> vePairList) {
-        stringBuilder.append("[");
-        boolean fst = true;
-        for (Pair<LogicalVariable, Mutable<ILogicalExpression>> ve : vePairList) {
-            if (fst) {
-                fst = false;
-            } else {
-                stringBuilder.append("; ");
+    private void appendSchema(AbstractLogicalOperator op, boolean show) {
+        if (show) {
+            stringBuilder.append("\\nSchema: ");
+            final List<LogicalVariable> schema = op.getSchema();
+            stringBuilder.append(schema == null ? "null" : schema);
+        }
+    }
+
+    private void appendAnnotations(AbstractLogicalOperator op, boolean show) {
+        if (show) {
+            final Map<String, Object> annotations = op.getAnnotations();
+            if (!annotations.isEmpty()) {
+                stringBuilder.append("\\nAnnotations: ").append(annotations);
             }
-            if (ve.first != null) {
-                stringBuilder.append(ve.first).append(" := ").append(ve.second);
-            } else {
-                stringBuilder.append(ve.second.getValue().toString());
+        }
+    }
+
+    private void appendPhysicalOperatorInfo(AbstractLogicalOperator op, boolean show) {
+        IPhysicalOperator physicalOp = op.getPhysicalOperator();
+        stringBuilder.append("\\n").append(physicalOp == null ? "null" : physicalOp.toString().trim());
+        stringBuilder.append(", Exec: ").append(op.getExecutionMode());
+        if (show) {
+            IPhysicalPropertiesVector properties = physicalOp == null ? null : physicalOp.getDeliveredProperties();
+            List<ILocalStructuralProperty> localProp = properties == null ? null : properties.getLocalProperties();
+            IPartitioningProperty partitioningProp = properties == null ? null : properties.getPartitioningProperty();
+            if (localProp != null) {
+                stringBuilder.append("\\nProperties in each partition: [");
+                for (ILocalStructuralProperty property : localProp) {
+                    if (property == null) {
+                        stringBuilder.append("null, ");
+                    } else if (property.getPropertyType() == LOCAL_ORDER_PROPERTY) {
+                        stringBuilder.append("ordered by ");
+                    } else if (property.getPropertyType() == LOCAL_GROUPING_PROPERTY) {
+                        stringBuilder.append("group by ");
+                    }
+                    stringBuilder.append(property).append(", ");
+                }
+                stringBuilder.append("]");
+            }
+
+            if (partitioningProp != null) {
+                stringBuilder.append("\\n").append(partitioningProp.getPartitioningType()).append(":");
+                INodeDomain nodeDomain = partitioningProp.getNodeDomain();
+                stringBuilder.append("\\n ");
+                if (nodeDomain != null && nodeDomain.cardinality() != null) {
+                    stringBuilder.append(nodeDomain.cardinality()).append(" partitions. ");
+                }
+                switch (partitioningProp.getPartitioningType()) {
+                    case BROADCAST:
+                        stringBuilder.append("Data is broadcast to partitions.");
+                        break;
+                    case RANDOM:
+                        stringBuilder.append("Data is randomly partitioned.");
+                        break;
+                    case ORDERED_PARTITIONED:
+                        stringBuilder.append("Data is orderly partitioned via a range.");
+                        break;
+                    case UNORDERED_PARTITIONED:
+                        stringBuilder.append("Data is hash partitioned.");
+                        break;
+                    case UNPARTITIONED:
+                        stringBuilder.append("Data is in one place.");
+                }
+                if (nodeDomain instanceof DefaultNodeGroupDomain) {
+                    DefaultNodeGroupDomain nd = (DefaultNodeGroupDomain) nodeDomain;
+                    stringBuilder.append("\\n").append(nd);
+                }
             }
         }
-        stringBuilder.append("]");
+
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
index 6f7f86a..cdab2f4 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/EnforceStructuralPropertiesRule.java
@@ -21,9 +21,11 @@ package org.apache.hyracks.algebricks.rewriter.rules;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.UUID;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
@@ -40,28 +42,41 @@ import org.apache.hyracks.algebricks.core.algebra.base.LogicalOperatorTag;
 import org.apache.hyracks.algebricks.core.algebra.base.LogicalVariable;
 import org.apache.hyracks.algebricks.core.algebra.base.OperatorAnnotations;
 import org.apache.hyracks.algebricks.core.algebra.base.PhysicalOperatorTag;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractFunctionCallExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AbstractLogicalExpression;
+import org.apache.hyracks.algebricks.core.algebra.expressions.AggregateFunctionCallExpression;
 import org.apache.hyracks.algebricks.core.algebra.expressions.VariableReferenceExpression;
+import org.apache.hyracks.algebricks.core.algebra.functions.FunctionIdentifier;
+import org.apache.hyracks.algebricks.core.algebra.functions.IFunctionInfo;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractOperatorWithNestedPlans;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AggregateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.DistinctOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.ExchangeOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ForwardOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.OrderOperator.IOrder.OrderKind;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.ReplicateOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.FDsAndEquivClassesVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractPreSortedDistinctByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractPreclusteredGroupByPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.AbstractStableSortPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.AggregatePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.BroadcastExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.ExternalGroupByPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ForwardPOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.HashPartitionMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.InMemoryStableSortPOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.OneToOneExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RandomPartitionExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.RangePartitionMergeExchangePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.ReplicatePOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.physical.SequentialMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.SortMergeExchangePOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.physical.StableSortPOperator;
 import org.apache.hyracks.algebricks.core.algebra.prettyprint.LogicalOperatorPrettyPrintVisitor;
@@ -90,13 +105,20 @@ import org.apache.hyracks.algebricks.core.rewriter.base.IAlgebraicRewriteRule;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
 import org.apache.hyracks.algebricks.rewriter.util.PhysicalOptimizationsUtil;
 import org.apache.hyracks.api.exceptions.SourceLocation;
-import org.apache.hyracks.dataflow.common.data.partition.range.IRangeMap;
+import org.apache.hyracks.dataflow.common.data.partition.range.RangeMap;
 
 public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
 
     private static final String HASH_MERGE = "hash_merge";
     private static final String TRUE_CONSTANT = "true";
     private PhysicalOptimizationConfig physicalOptimizationConfig;
+    private final FunctionIdentifier rangeMapFunction;
+    private final FunctionIdentifier localSamplingFun;
+
+    public EnforceStructuralPropertiesRule(FunctionIdentifier rangeMapFunction, FunctionIdentifier localSamplingFun) {
+        this.rangeMapFunction = rangeMapFunction;
+        this.localSamplingFun = localSamplingFun;
+    }
 
     @Override
     public boolean rewritePost(Mutable<ILogicalOperator> opRef, IOptimizationContext context)
@@ -204,6 +226,7 @@ public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
 
         boolean changed = false;
         AbstractLogicalOperator op = (AbstractLogicalOperator) opRef.getValue();
+
         optimizeUsingConstraintsAndEquivClasses(op);
         PhysicalRequirements pr = op.getRequiredPhysicalPropertiesForChildren(required, context);
         IPhysicalPropertiesVector[] reqdProperties = null;
@@ -214,26 +237,19 @@ public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
 
         // compute properties and figure out the domain
         INodeDomain childrenDomain = null;
-        {
-            int j = 0;
-            for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
-                AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
-                // recursive call
-                if (physOptimizeOp(childRef, reqdProperties[j], nestedPlan, context)) {
-                    changed = true;
-                }
-                child.computeDeliveredPhysicalProperties(context);
-                IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
-                if (childrenDomain == null) {
-                    childrenDomain = delivered.getPartitioningProperty().getNodeDomain();
-                } else {
-                    INodeDomain dom2 = delivered.getPartitioningProperty().getNodeDomain();
-                    if (!childrenDomain.sameAs(dom2)) {
-                        childrenDomain = context.getComputationNodeDomain();
-                    }
-                }
-                j++;
+        int j = 0;
+        for (Mutable<ILogicalOperator> childRef : op.getInputs()) {
+            AbstractLogicalOperator child = (AbstractLogicalOperator) childRef.getValue();
+            changed |= physOptimizeOp(childRef, reqdProperties[j], nestedPlan, context);
+            child.computeDeliveredPhysicalProperties(context);
+            IPhysicalPropertiesVector delivered = child.getDeliveredPhysicalProperties();
+            INodeDomain childDomain = delivered.getPartitioningProperty().getNodeDomain();
+            if (childrenDomain == null) {
+                childrenDomain = delivered.getPartitioningProperty().getNodeDomain();
+            } else if (!childrenDomain.sameAs(childDomain)) {
+                childrenDomain = context.getComputationNodeDomain();
             }
+            j++;
         }
 
         if (reqdProperties != null) {
@@ -252,7 +268,7 @@ public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
         int startChildIndex = getStartChildIndex(op, pr, nestedPlan, context);
         IPartitioningProperty firstDeliveredPartitioning = null;
         // Enforce data properties in a top-down manner.
-        for (int j = 0; j < op.getInputs().size(); j++) {
+        for (j = 0; j < op.getInputs().size(); j++) {
             // Starts from a partitioning-compatible child if any to loop over all children.
             int childIndex = (j + startChildIndex) % op.getInputs().size();
             IPhysicalPropertiesVector requiredProperty = reqdProperties[childIndex];
@@ -555,6 +571,17 @@ public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
         return new MutableObject<ILogicalOperator>(oo);
     }
 
+    /**
+     * Adds exchange operators (connectors) between {@code op} & its child at index {@code childIdx}.
+     * @param op the parent operator that is requiring a specific kind of connector at its child
+     * @param i the child index where we want to have the connector
+     * @param pp the required partitioning property at that child (i.e. the required connector)
+     * @param required the physical properties required at that child (partitioning + local properties)
+     * @param deliveredByChild the physical properties delivered by that child (partitioning + local properties)
+     * @param domain the destination domain of nodes that we want the connector to connect to
+     * @param context {@link IOptimizationContext}
+     * @throws AlgebricksException
+     */
     private void addPartitioningEnforcers(ILogicalOperator op, int i, IPartitioningProperty pp,
             IPhysicalPropertiesVector required, IPhysicalPropertiesVector deliveredByChild, INodeDomain domain,
             IOptimizationContext context) throws AlgebricksException {
@@ -562,52 +589,15 @@ public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
             IPhysicalOperator pop;
             switch (pp.getPartitioningType()) {
                 case UNPARTITIONED: {
-                    List<OrderColumn> ordCols = computeOrderColumns(deliveredByChild);
-                    if (ordCols.isEmpty()) {
-                        pop = new RandomMergeExchangePOperator();
-                    } else {
-                        if (op.getAnnotations().containsKey(OperatorAnnotations.USE_RANGE_CONNECTOR)) {
-                            IRangeMap rangeMap =
-                                    (IRangeMap) op.getAnnotations().get(OperatorAnnotations.USE_RANGE_CONNECTOR);
-                            pop = new RangePartitionMergeExchangePOperator(ordCols, domain, rangeMap);
-                        } else {
-                            OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
-                            sortColumns = ordCols.toArray(sortColumns);
-                            pop = new SortMergeExchangePOperator(sortColumns);
-                        }
-                    }
+                    pop = createMergingConnector(op, domain, deliveredByChild);
                     break;
                 }
                 case UNORDERED_PARTITIONED: {
-                    List<LogicalVariable> varList = new ArrayList<>(((UnorderedPartitionedProperty) pp).getColumnSet());
-                    String hashMergeHint = (String) context.getMetadataProvider().getConfig().get(HASH_MERGE);
-                    if (hashMergeHint == null || !hashMergeHint.equalsIgnoreCase(TRUE_CONSTANT)) {
-                        pop = new HashPartitionExchangePOperator(varList, domain);
-                        break;
-                    }
-                    List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
-                    List<ILocalStructuralProperty> reqdLocals = required.getLocalProperties();
-                    boolean propWasSet = false;
-                    pop = null;
-                    if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
-                        AbstractLogicalOperator c = (AbstractLogicalOperator) op.getInputs().get(i).getValue();
-                        Map<LogicalVariable, EquivalenceClass> ecs = context.getEquivalenceClassMap(c);
-                        List<FunctionalDependency> fds = context.getFDList(c);
-                        if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
-                            List<OrderColumn> orderColumns =
-                                    getOrderColumnsFromGroupingProperties(reqdLocals, cldLocals);
-                            pop = new HashPartitionMergeExchangePOperator(orderColumns, varList, domain);
-                            propWasSet = true;
-                        }
-                    }
-                    if (!propWasSet) {
-                        pop = new HashPartitionExchangePOperator(varList, domain);
-                    }
+                    pop = createHashConnector(context, deliveredByChild, domain, required, pp, i, op);
                     break;
                 }
                 case ORDERED_PARTITIONED: {
-                    pop = new RangePartitionExchangePOperator(((OrderedPartitionedProperty) pp).getOrderColumns(),
-                            domain, null);
+                    pop = createRangePartitionerConnector((AbstractLogicalOperator) op, domain, pp, i, context);
                     break;
                 }
                 case BROADCAST: {
@@ -640,6 +630,264 @@ public class EnforceStructuralPropertiesRule implements IAlgebraicRewriteRule {
         }
     }
 
+    private IPhysicalOperator createMergingConnector(ILogicalOperator parentOp, INodeDomain domain,
+            IPhysicalPropertiesVector deliveredByChild) {
+        IPhysicalOperator mergingConnector;
+        List<OrderColumn> ordCols = computeOrderColumns(deliveredByChild);
+        if (ordCols.isEmpty()) {
+            IPartitioningProperty partitioningDeliveredByChild = deliveredByChild.getPartitioningProperty();
+            if (partitioningDeliveredByChild.getPartitioningType() == PartitioningType.ORDERED_PARTITIONED) {
+                mergingConnector = new SequentialMergeExchangePOperator();
+            } else {
+                mergingConnector = new RandomMergeExchangePOperator();
+            }
+        } else {
+            if (parentOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
+                RangeMap rangeMap = (RangeMap) parentOp.getAnnotations().get(OperatorAnnotations.USE_STATIC_RANGE);
+                mergingConnector = new RangePartitionMergeExchangePOperator(ordCols, domain, rangeMap);
+            } else {
+                OrderColumn[] sortColumns = new OrderColumn[ordCols.size()];
+                sortColumns = ordCols.toArray(sortColumns);
+                mergingConnector = new SortMergeExchangePOperator(sortColumns);
+            }
+        }
+        return mergingConnector;
+    }
+
+    private IPhysicalOperator createHashConnector(IOptimizationContext ctx, IPhysicalPropertiesVector deliveredByChild,
+            INodeDomain domain, IPhysicalPropertiesVector requiredAtChild, IPartitioningProperty rqdPartitioning,
+            int childIndex, ILogicalOperator parentOp) {
+        IPhysicalOperator hashConnector;
+        List<LogicalVariable> vars = new ArrayList<>(((UnorderedPartitionedProperty) rqdPartitioning).getColumnSet());
+        String hashMergeHint = (String) ctx.getMetadataProvider().getConfig().get(HASH_MERGE);
+        if (hashMergeHint == null || !hashMergeHint.equalsIgnoreCase(TRUE_CONSTANT)) {
+            hashConnector = new HashPartitionExchangePOperator(vars, domain);
+            return hashConnector;
+        }
+        List<ILocalStructuralProperty> cldLocals = deliveredByChild.getLocalProperties();
+        List<ILocalStructuralProperty> reqdLocals = requiredAtChild.getLocalProperties();
+        boolean propWasSet = false;
+        hashConnector = null;
+        if (reqdLocals != null && cldLocals != null && allAreOrderProps(cldLocals)) {
+            AbstractLogicalOperator c = (AbstractLogicalOperator) parentOp.getInputs().get(childIndex).getValue();
+            Map<LogicalVariable, EquivalenceClass> ecs = ctx.getEquivalenceClassMap(c);
+            List<FunctionalDependency> fds = ctx.getFDList(c);
+            if (PropertiesUtil.matchLocalProperties(reqdLocals, cldLocals, ecs, fds)) {
+                List<OrderColumn> orderColumns = getOrderColumnsFromGroupingProperties(reqdLocals, cldLocals);
+                hashConnector = new HashPartitionMergeExchangePOperator(orderColumns, vars, domain);
+                propWasSet = true;
+            }
+        }
+        if (!propWasSet) {
+            hashConnector = new HashPartitionExchangePOperator(vars, domain);
+        }
+        return hashConnector;
+    }
+
+    /**
+     * Creates a range-based exchange operator.
+     * @param parentOp the operator requiring range-based partitioner to have input tuples repartitioned using a range
+     * @param domain the target node domain of the range-based partitioner
+     * @param requiredPartitioning {@see OrderedPartitionedProperty}
+     * @param childIndex the index of the child at which the required partitioning is needed
+     * @param ctx optimization context
+     * @return a range-based exchange operator
+     * @throws AlgebricksException
+     */
+    private IPhysicalOperator createRangePartitionerConnector(AbstractLogicalOperator parentOp, INodeDomain domain,
+            IPartitioningProperty requiredPartitioning, int childIndex, IOptimizationContext ctx)
+            throws AlgebricksException {
+        // options for range partitioning: 1. static range map, 2. dynamic range map computed at run time
+        List<OrderColumn> partitioningColumns = ((OrderedPartitionedProperty) requiredPartitioning).getOrderColumns();
+        if (parentOp.getAnnotations().containsKey(OperatorAnnotations.USE_STATIC_RANGE)) {
+            // TODO(ali): static range map implementation should be fixed to require ORDERED_PARTITION and come here.
+            RangeMap rangeMap = (RangeMap) parentOp.getAnnotations().get(OperatorAnnotations.USE_STATIC_RANGE);
+            return new RangePartitionExchangePOperator(partitioningColumns, domain, rangeMap);
+        } else {
+            return createDynamicRangePartitionExchangePOperator(parentOp, ctx, domain, partitioningColumns, childIndex);
+        }
+    }
+
+    private IPhysicalOperator createDynamicRangePartitionExchangePOperator(AbstractLogicalOperator parentOp,
+            IOptimizationContext ctx, INodeDomain targetDomain, List<OrderColumn> partitioningColumns, int childIndex)
+            throws AlgebricksException {
+        SourceLocation sourceLoc = parentOp.getSourceLocation();
+        // #1. create the replicate operator and add it above the source op feeding parent operator
+        ReplicateOperator replicateOp = createReplicateOperator(parentOp.getInputs().get(childIndex), ctx, sourceLoc);
+
+        // these two exchange ops are needed so that the parents of replicate stay the same during later optimizations.
+        // This is because replicate operator has references to its parents. If any later optimizations add new parents,
+        // then replicate would still point to the old ones.
+        MutableObject<ILogicalOperator> replicateOpRef = new MutableObject<>(replicateOp);
+        ExchangeOperator exchToLocalAgg = createOneToOneExchangeOp(replicateOpRef, ctx);
+        ExchangeOperator exchToForward = createOneToOneExchangeOp(replicateOpRef, ctx);
+        MutableObject<ILogicalOperator> exchToLocalAggRef = new MutableObject<>(exchToLocalAgg);
+        MutableObject<ILogicalOperator> exchToForwardRef = new MutableObject<>(exchToForward);
+
+        // add the exchange--to-forward at output 0, the exchange-to-local-aggregate at output 1
+        replicateOp.getOutputs().add(exchToForwardRef);
+        replicateOp.getOutputs().add(exchToLocalAggRef);
+        // materialize the data to be able to re-read the data again after sampling is done
+        replicateOp.getOutputMaterializationFlags()[0] = true;
+
+        // #2. create the aggregate operators and their sampling functions
+        // $$samplingResultVar = local_samplingFun($$partitioning_column)
+        // $$rangeMapResultVar = global_rangeMapFun($$samplingResultVar)
+        List<LogicalVariable> samplingResultVar = new ArrayList<>(1);
+        List<LogicalVariable> rangeMapResultVar = new ArrayList<>(1);
+        List<Mutable<ILogicalExpression>> samplingFun = new ArrayList<>(1);
+        List<Mutable<ILogicalExpression>> rangeMapFun = new ArrayList<>(1);
+
+        createAggregateFunction(ctx, samplingResultVar, samplingFun, rangeMapResultVar, rangeMapFun,
+                targetDomain.cardinality(), partitioningColumns, sourceLoc);
+
+        AggregateOperator localAggOp =
+                createAggregate(samplingResultVar, false, samplingFun, exchToLocalAggRef, ctx, sourceLoc);
+        MutableObject<ILogicalOperator> localAgg = new MutableObject<>(localAggOp);
+        AggregateOperator globalAggOp = createAggregate(rangeMapResultVar, true, rangeMapFun, localAgg, ctx, sourceLoc);
+        MutableObject<ILogicalOperator> globalAgg = new MutableObject<>(globalAggOp);
+
+        // #3. create the forward operator
+        String rangeMapKey = UUID.randomUUID().toString();
+        LogicalVariable rangeMapVar = rangeMapResultVar.get(0);
+        ForwardOperator forward = createForward(rangeMapKey, rangeMapVar, exchToForwardRef, globalAgg, ctx, sourceLoc);
+        MutableObject<ILogicalOperator> forwardRef = new MutableObject<>(forward);
+
+        // replace the old input of parentOp requiring the range partitioning with the new forward op
+        parentOp.getInputs().set(childIndex, forwardRef);
+        parentOp.recomputeSchema();
+        ctx.computeAndSetTypeEnvironmentForOperator(parentOp);
+
+        return new RangePartitionExchangePOperator(partitioningColumns, rangeMapKey, targetDomain);
+    }
+
+    private static ReplicateOperator createReplicateOperator(Mutable<ILogicalOperator> inputOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        ReplicateOperator replicateOperator = new ReplicateOperator(2);
+        replicateOperator.setPhysicalOperator(new ReplicatePOperator());
+        replicateOperator.setSourceLocation(sourceLocation);
+        replicateOperator.getInputs().add(inputOperator);
+        OperatorManipulationUtil.setOperatorMode(replicateOperator);
+        replicateOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(replicateOperator);
+        return replicateOperator;
+    }
+
+    /**
+     * Creates the sampling expressions and embeds them in {@code localAggFunctions} & {@code globalAggFunctions}. Also,
+     * creates the variables which will hold the result of each one.
+     * {@code localResultVariables},{@code localAggFunctions},{@code globalResultVariables} & {@code globalAggFunctions}
+     * will be used when creating the corresponding aggregate operators.
+     * @param context used to get new variables which will be assigned the samples & the range map
+     * @param localResultVariables the variable to which the stats (e.g. samples) info is assigned
+     * @param localAggFunctions the local sampling expression is added to this list
+     * @param globalResultVariables the variable to which the range map is assigned
+     * @param globalAggFunctions the expression generating a range map is added to this list
+     * @param numPartitions passed to the expression generating a range map to know how many split points are needed
+     * @param partFields the fields based on which the partitioner partitions the tuples, also sampled fields
+     * @param sourceLocation source location
+     */
+    private void createAggregateFunction(IOptimizationContext context, List<LogicalVariable> localResultVariables,
+            List<Mutable<ILogicalExpression>> localAggFunctions, List<LogicalVariable> globalResultVariables,
+            List<Mutable<ILogicalExpression>> globalAggFunctions, int numPartitions, List<OrderColumn> partFields,
+            SourceLocation sourceLocation) {
+        // prepare the arguments of the local sampling function: sampled fields
+        List<Mutable<ILogicalExpression>> sampledFields = new ArrayList<>(partFields.size());
+        partFields.forEach(f -> {
+            AbstractLogicalExpression sampledField = new VariableReferenceExpression(f.getColumn());
+            sampledField.setSourceLocation(sourceLocation);
+            sampledFields.add(new MutableObject<>(sampledField));
+        });
+
+        // local info
+        IFunctionInfo samplingFun = context.getMetadataProvider().lookupFunction(localSamplingFun);
+        AbstractFunctionCallExpression samplingExp =
+                new AggregateFunctionCallExpression(samplingFun, false, sampledFields);
+        samplingExp.setSourceLocation(sourceLocation);
+        LogicalVariable samplingResultVar = context.newVar();
+        localResultVariables.add(samplingResultVar);
+        localAggFunctions.add(new MutableObject<>(samplingExp));
+        Object[] samplingParam = { context.getPhysicalOptimizationConfig().getSortSamples() };
+        samplingExp.setOpaqueParameters(samplingParam);
+
+        // prepare the argument of the global range map generator function: the result of the local function
+        List<Mutable<ILogicalExpression>> arg = new ArrayList<>(1);
+        AbstractLogicalExpression samplingResultVarExp = new VariableReferenceExpression(samplingResultVar);
+        samplingResultVarExp.setSourceLocation(sourceLocation);
+        arg.add(new MutableObject<>(samplingResultVarExp));
+
+        // global info
+        IFunctionInfo rangeMapFun = context.getMetadataProvider().lookupFunction(rangeMapFunction);
+        AbstractFunctionCallExpression rangeMapExp = new AggregateFunctionCallExpression(rangeMapFun, true, arg);
+        rangeMapExp.setSourceLocation(sourceLocation);
+        globalResultVariables.add(context.newVar());
+        globalAggFunctions.add(new MutableObject<>(rangeMapExp));
+
+        int i = 0;
+        boolean[] ascendingFlags = new boolean[partFields.size()];
+        for (OrderColumn column : partFields) {
+            ascendingFlags[i] = column.getOrder() == OrderOperator.IOrder.OrderKind.ASC;
+            i++;
+        }
+        rangeMapExp.setOpaqueParameters(new Object[] { numPartitions, ascendingFlags });
+    }
+
+    /**
+     * Creates an aggregate operator. $$resultVariables = expressions()
+     * @param resultVariables the variables which stores the result of the aggregation
+     * @param isGlobal whether the aggregate operator is a global or local one
+     * @param expressions the aggregation functions desired
+     * @param inputOperator the input op that is feeding the aggregate operator
+     * @param context optimization context
+     * @param sourceLocation source location
+     * @return an aggregate operator with the specified information
+     * @throws AlgebricksException when there is error setting the type environment of the newly created aggregate op
+     */
+    private static AggregateOperator createAggregate(List<LogicalVariable> resultVariables, boolean isGlobal,
+            List<Mutable<ILogicalExpression>> expressions, MutableObject<ILogicalOperator> inputOperator,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        AggregateOperator aggregateOperator = new AggregateOperator(resultVariables, expressions);
+        aggregateOperator.setPhysicalOperator(new AggregatePOperator());
+        aggregateOperator.setSourceLocation(sourceLocation);
+        aggregateOperator.getInputs().add(inputOperator);
+        aggregateOperator.setGlobal(isGlobal);
+        if (!isGlobal) {
+            aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.LOCAL);
+        } else {
+            aggregateOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        }
+        aggregateOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(aggregateOperator);
+        return aggregateOperator;
+    }
+
+    private static ExchangeOperator createOneToOneExchangeOp(MutableObject<ILogicalOperator> inputOperator,
+            IOptimizationContext context) throws AlgebricksException {
+        ExchangeOperator exchangeOperator = new ExchangeOperator();
+        exchangeOperator.setPhysicalOperator(new OneToOneExchangePOperator());
+        exchangeOperator.getInputs().add(inputOperator);
+        exchangeOperator.setExecutionMode(AbstractLogicalOperator.ExecutionMode.PARTITIONED);
+        exchangeOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(exchangeOperator);
+        return exchangeOperator;
+    }
+
+    private static ForwardOperator createForward(String rangeMapKey, LogicalVariable rangeMapVariable,
+            MutableObject<ILogicalOperator> exchangeOpFromReplicate, MutableObject<ILogicalOperator> globalAggInput,
+            IOptimizationContext context, SourceLocation sourceLocation) throws AlgebricksException {
+        AbstractLogicalExpression rangeMapExpression = new VariableReferenceExpression(rangeMapVariable);
+        rangeMapExpression.setSourceLocation(sourceLocation);
+        ForwardOperator forwardOperator = new ForwardOperator(rangeMapKey, new MutableObject<>(rangeMapExpression));
+        forwardOperator.setSourceLocation(sourceLocation);
+        forwardOperator.setPhysicalOperator(new ForwardPOperator());
+        forwardOperator.getInputs().add(exchangeOpFromReplicate);
+        forwardOperator.getInputs().add(globalAggInput);
+        OperatorManipulationUtil.setOperatorMode(forwardOperator);
+        forwardOperator.recomputeSchema();
+        context.computeAndSetTypeEnvironmentForOperator(forwardOperator);
+        return forwardOperator;
+    }
+
     private boolean allAreOrderProps(List<ILocalStructuralProperty> cldLocals) {
         for (ILocalStructuralProperty lsp : cldLocals) {
             if (lsp.getPropertyType() != PropertyType.LOCAL_ORDER_PROPERTY) {


[12/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..0a9089e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/word-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/word-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/word-contains_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-basic/word-contains_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..f145eac
--- /dev/null
+++ 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
@@ -0,0 +1,165 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$63(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$63(ASC)]  |PARTITIONED|
+              -- FORWARD  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                  -- REPLICATE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                      -- PRE_CLUSTERED_GROUP_BY[$$53]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- STREAM_SELECT  |LOCAL|
+                                    -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$69][$$53]  |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  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$53]  |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  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- BTREE_SEARCH  |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[$$53]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$53(ASC), $$54(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$53]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$69][$$53]  |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  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$53]  |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  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- STABLE_SORT [$$77(ASC)]  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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  |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  |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|
+                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                          -- BTREE_SEARCH  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- ASSIGN  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..428c860
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_01_ps.plan
@@ -0,0 +1,50 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |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  |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|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |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  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_02_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |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|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_03_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |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|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..04cdee5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/inverted-index-join/ngram-contains_04_ps.plan
@@ -0,0 +1,54 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |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|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..c95c27f
--- /dev/null
+++ 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
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$55(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$55(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_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$67(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |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_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$67(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..35e1ec6
--- /dev/null
+++ 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
@@ -0,0 +1,83 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$60(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$60(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|
+                              }
+                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                          -- STABLE_SORT [$$52(ASC), $$53(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$52]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- STREAM_SELECT  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ASSIGN  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- RTREE_SEARCH  |PARTITIONED|
+                                                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ASSIGN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- BTREE_SEARCH  |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_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STABLE_SORT [$$72(ASC)]  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- RTREE_SEARCH  |PARTITIONED|
+                                                                  -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ASSIGN  |PARTITIONED|
+                                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                  -- ASSIGN  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..101441e
--- /dev/null
+++ 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
@@ -0,0 +1,56 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$32(ASC), $$33(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$32(ASC), $$33(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 [$$30][$$31]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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 [$$30][$$31]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..101441e
--- /dev/null
+++ 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
@@ -0,0 +1,56 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$32(ASC), $$33(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$32(ASC), $$33(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 [$$30][$$31]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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 [$$30][$$31]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$30]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..3da4637
--- /dev/null
+++ 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
@@ -0,0 +1,56 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$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 [$$31][$$32]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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 [$$31][$$32]  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$31]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                    -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2eb5d36
--- /dev/null
+++ 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
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |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  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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  |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  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/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
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..2eb5d36
--- /dev/null
+++ 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
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$34(ASC), $$35(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$34(ASC), $$35(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |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  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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  |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  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..16c9194
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-04_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |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  |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  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..16c9194
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-05_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |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  |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  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..07710ba
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/open-index-non-enforced/btree-index-non-enforced/btree-index-non-enforced-06_ps.plan
@@ -0,0 +1,48 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- BTREE_SEARCH  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- 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  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$22(ASC)]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|


[19/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
new file mode 100644
index 0000000..5b88828
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where (t1.nested.countA /*+ indexnl */  = t2.nested.countB)
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
new file mode 100644
index 0000000..bb50b07
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountAIx  on TweetMessages (nested.countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where (t1.nested.countA /*+ indexnl */  = t2.nested.countB)
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
new file mode 100644
index 0000000..dbf7c40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where ((t1.nested.countA /*+ indexnl */  = t2.nested.countB) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
new file mode 100644
index 0000000..df41b65
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgCountAIx  on TweetMessages (nested.countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (nested.countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'count1':t1.nested.countA,'t2info':(
+        select element {'tweetid2':t2.nested.tweetid,'count2':t2.nested.countB}
+        from  TweetMessages as t2
+        where ((t1.nested.countA /*+ indexnl */  = t2.nested.countB) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..1d22a19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains-panic_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Mu')
+order by o.nested.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..7e98edf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/ngram-contains_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index ngram_index  on DBLP (nested.title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..b6407d5
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-basic/word-contains_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPTypetmp as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create type test.DBLPType as
+ closed {
+  nested : DBLPTypetmp
+};
+
+create  dataset DBLP(DBLPType) primary key nested.id;
+
+create  index keyword_index  on DBLP (nested.title:string?) type keyword enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.nested.title,'Multimedia')
+order by o.nested.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..0a55178
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index msgNgramIx  on TweetMessages (nested.`message-text`:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.nested.tweetid,'topics':t1.nested.`message-text`},'similar-tweets':(
+        select element {'id':t2.nested.tweetid,'topics':t2.nested.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.nested.`message-text`,t2.nested.`message-text`,7)
+        where (sim[0] and (t2.nested.tweetid != t1.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.nested.tweetid > test.bigint('240'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
new file mode 100644
index 0000000..fe6c65d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
new file mode 100644
index 0000000..955f38c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_02_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  CSX as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
new file mode 100644
index 0000000..eb6fe2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_03_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
new file mode 100644
index 0000000..a924e19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/inverted-index-join/ngram-contains_04_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index_DBLP  on DBLP (title:string?) type ngram (3) enforced;
+
+create  index ngram_index_CSX  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..19143d3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.nested.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..0471760
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/nested-open-index/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageNestedType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create type test.TweetMessageType as
+{
+  nested : TweetMessageNestedType
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key nested.tweetid;
+
+create  index twmSndLocIx  on TweetMessages (nested.`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (nested.countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (nested.countB) type btree;
+
+create  index msgTextIx  on TweetMessages (nested.`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.nested.tweetid,'loc1':t1.nested.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.nested.tweetid,'loc2':t2.nested.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.nested.`sender-location`,n) and (t1.nested.tweetid != t2.nested.tweetid))
+        order by t2.nested.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.nested.`sender-location`,0.5)
+where (t1.nested.tweetid < test.bigint('10'))
+order by t1.nested.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.sqlpp
new file mode 100644
index 0000000..2514206
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_1_ps.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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where (t1.countA /*+ indexnl */  = t2.countB)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
new file mode 100644
index 0000000..abe7d59
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_2_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountAIx  on TweetMessages (countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where (t1.countA /*+ indexnl */  = t2.countB)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.sqlpp
new file mode 100644
index 0000000..1f0c190
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_1_ps.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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
new file mode 100644
index 0000000..fd29e2e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_2_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary btree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgCountAIx  on TweetMessages (countA:integer?) type btree enforced;
+
+create  index msgCountBIx  on TweetMessages (countB:integer?) type btree enforced;
+
+write output to asterix_nc1:"rttest/btree-index-join_leftouterjoin-probe-pidx-with-join-btree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'count1':t1.countA,'t2info':(
+        select element {'tweetid2':t2.tweetid,'count2':t2.countB}
+        from  TweetMessages as t2
+        where ((t1.countA /*+ indexnl */  = t2.countB) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.sqlpp
new file mode 100644
index 0000000..dbb430a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains-panic_ps.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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains-panic.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Mu')
+order by o.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.sqlpp
new file mode 100644
index 0000000..060465d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/ngram-contains_ps.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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a selection query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_ngram-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.sqlpp
new file mode 100644
index 0000000..9caf6bd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-basic/word-contains_ps.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.
+ */
+/*
+ * Description    : Tests whether a keyword index is applied to optimize a selection query using the contains function.
+ *                  The index should *not* be applied (see below).
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index keyword_index  on DBLP (title:string?) type keyword enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-basic_word-contains.adm";
+set `compiler.sort.parallel` "true";
+
+select element o
+from  DBLP as o
+where test.contains(o.title,'Multimedia')
+order by o.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
new file mode 100644
index 0000000..2d9b15b
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/leftouterjoin-probe-pidx-with-join-edit-distance-check-idx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `sender-location` : point,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index msgNgramIx  on TweetMessages (`message-text`:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_leftouterjoin-probe-pidx-with-join-edit-distance-check_idx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweet':{'id':t1.tweetid,'topics':t1.`message-text`},'similar-tweets':(
+        select element {'id':t2.tweetid,'topics':t2.`message-text`}
+        from  TweetMessages as t2
+        with  sim as test.`edit-distance-check`(t1.`message-text`,t2.`message-text`,7)
+        where (sim[0] and (t2.tweetid != t1.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+where (t1.tweetid > test.bigint('240'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
new file mode 100644
index 0000000..fe6c65d
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+ closed {
+  id : integer,
+  csxid : string,
+  title : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
new file mode 100644
index 0000000..1cb8237
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_02_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+ closed {
+  id : integer,
+  dblpid : string,
+  title : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
new file mode 100644
index 0000000..eb6fe2c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_03_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  index ngram_index  on DBLP (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-03.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      DBLP as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
new file mode 100644
index 0000000..a924e19
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/inverted-index-join/ngram-contains_04_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description    : Tests whether an ngram_index is applied to optimize a join query using the contains function.
+ *                  The index should be applied.
+ * Success        : Yes
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.DBLPType as
+{
+  id : integer,
+  dblpid : string,
+  misc : string
+};
+
+create type test.CSXType as
+{
+  id : integer,
+  csxid : string,
+  authors : string,
+  misc : string
+};
+
+create  dataset DBLP(DBLPType) primary key id;
+
+create  dataset CSX(CSXType) primary key id;
+
+create  index ngram_index_DBLP  on DBLP (title:string?) type ngram (3) enforced;
+
+create  index ngram_index_CSX  on CSX (title:string?) type ngram (3) enforced;
+
+write output to asterix_nc1:"rttest/inverted-index-join_ngram-contains-04.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'title1':o1.title,'title2':o2.title}
+from  DBLP as o1,
+      CSX as o2
+where (test.contains(o1.title,o2.title) and (o1.id < o2.id))
+order by o1.id,o2.id
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
new file mode 100644
index 0000000..3fd898e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_01_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_01.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where test.`spatial-intersect`(t2.`sender-location`,n)
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
new file mode 100644
index 0000000..c59dba4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-enforced/rtree-index-join/leftouterjoin-probe-pidx-with-join-rtree-sidx_02_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : Test that left-outer-join may use two available indexes, one for primary index in prob subtree and another for secondary rtree index in index subtree.
+ * Issue        : 730, 741
+ * Expected Res : Success
+ * Date         : 8th May 2014
+ */
+
+drop  dataverse test if exists;
+create  dataverse test;
+
+use test;
+
+
+create type test.TwitterUserType as
+ closed {
+  `screen-name` : string,
+  lang : string,
+  `friends-count` : integer,
+  `statuses-count` : integer,
+  name : string,
+  `followers-count` : integer
+};
+
+create type test.TweetMessageType as
+{
+  tweetid : bigint,
+  user : TwitterUserType,
+  `send-time` : datetime,
+  `referred-topics` : {{string}},
+  `message-text` : string,
+  countA : integer,
+  countB : integer
+};
+
+create  dataset TweetMessages(TweetMessageType) primary key tweetid;
+
+create  index twmSndLocIx  on TweetMessages (`sender-location`:point?) type rtree enforced;
+
+create  index msgCountAIx  on TweetMessages (countA) type btree;
+
+create  index msgCountBIx  on TweetMessages (countB) type btree;
+
+create  index msgTextIx  on TweetMessages (`message-text`) type keyword;
+
+write output to asterix_nc1:"rttest/rtree-index-join_leftouterjoin-probe-pidx-with-join-rtree-sidx_02.adm";
+set `compiler.sort.parallel` "true";
+
+select element {'tweetid1':t1.tweetid,'loc1':t1.`sender-location`,'nearby-message':(
+        select element {'tweetid2':t2.tweetid,'loc2':t2.`sender-location`}
+        from  TweetMessages as t2
+        where (test.`spatial-intersect`(t2.`sender-location`,n) and (t1.tweetid != t2.tweetid))
+        order by t2.tweetid
+    )}
+from  TweetMessages as t1
+with  n as test.`create-circle`(t1.`sender-location`,0.5)
+where (t1.tweetid < test.bigint('10'))
+order by t1.tweetid
+;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
new file mode 100644
index 0000000..8c3f6dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-05_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : No index join because there's no hint and the probe type is unknown. p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-05.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.c_s = t2.c_s
+order by t1.c_x, t2.c_x;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
new file mode 100644
index 0000000..7ba14c9
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-06_ps.sqlpp
@@ -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.
+ */
+/*
+ * Description  : No index join because the probe type is unknown. p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-06.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+select t1.c_x as c1, t2.c_x as c2
+from  TestOpen1 as t1, TestOpen2 as t2
+where t1.c_s /*+ indexnl */ = t2.c_s
+order by t1.c_x, t2.c_x;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.sqlpp
new file mode 100644
index 0000000..403a75e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/open-index-non-enforced/btree-equi-join-non-enforced/btree-equi-join-non-enforced-07_ps.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.
+ */
+/*
+ * Description  : No index join because there's no hint. p_sort enabled.
+ * Expected Res : Success
+ * Date         : 20 Jun 2017
+ */
+drop dataverse test if exists;
+create dataverse test;
+use test;
+
+write output to asterix_nc1:"rttest/btree-equi-join-non-enforced_btree-equi-join-non-enforced-07.adm";
+
+create type TestOpenType as open {
+  c_id: int64
+};
+
+create dataset TestOpen1(TestOpenType) primary key c_id;
+create dataset TestOpen2(TestOpenType) primary key c_id;
+create index idx_t2_s on TestOpen2(c_s:string);
+create index idx_t2_i64 on TestOpen2(c_i64:int64);
+create index idx_t2_i8 on TestOpen2(c_i8:int8);
+create index idx_t2_d on TestOpen2(c_d:double);
+
+set `compiler.sort.parallel` "true";
+
+
+select t1.c_x as c1, t2.c_x as c2
+from TestOpen1 as t1, TestOpen2 as t2
+where to_string(t1.c_s) = t2.c_s
+order by t1.c_x, t2.c_x;
\ No newline at end of file


[07/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_seq_merge/p_sort_seq_merge.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_seq_merge/p_sort_seq_merge.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_seq_merge/p_sort_seq_merge.3.adm
new file mode 100644
index 0000000..beb05c3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_seq_merge/p_sort_seq_merge.3.adm
@@ -0,0 +1 @@
+[ [ { "v": { "id": 1, "f1": 40 } }, { "v": { "id": 2, "f1": 40 } }, { "v": { "id": 3, "f1": 40 } }, { "v": { "id": 7, "f1": 40 } }, { "v": { "id": 11, "f1": 40 } }, { "v": { "id": 15, "f1": 40 } }, { "v": { "id": 4, "f1": 41 } }, { "v": { "id": 8, "f1": 41 } }, { "v": { "id": 9, "f1": 41 } }, { "v": { "id": 12, "f1": 41 } }, { "v": { "id": 14, "f1": 41 } }, { "v": { "id": 18, "f1": 41 } }, { "v": { "id": 5, "f1": 42 } }, { "v": { "id": 6, "f1": 42 } }, { "v": { "id": 10, "f1": 42 } }, { "v": { "id": 13, "f1": 42 } }, { "v": { "id": 16, "f1": 42 } }, { "v": { "id": 17, "f1": 42 } } ] ]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm
index f2090a8..ae73d82 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/prefix-search/prefix-search.1.adm
@@ -1,105 +1,105 @@
 { "l_linenumber": 1, "l_l_orderkey": 1 }
 { "l_linenumber": 1, "l_l_orderkey": 2 }
+{ "l_linenumber": 1, "l_l_orderkey": 3 }
 { "l_linenumber": 1, "l_l_orderkey": 4 }
+{ "l_linenumber": 1, "l_l_orderkey": 5 }
+{ "l_linenumber": 1, "l_l_orderkey": 6 }
+{ "l_linenumber": 1, "l_l_orderkey": 7 }
+{ "l_linenumber": 1, "l_l_orderkey": 32 }
+{ "l_linenumber": 1, "l_l_orderkey": 33 }
+{ "l_linenumber": 1, "l_l_orderkey": 34 }
 { "l_linenumber": 1, "l_l_orderkey": 35 }
+{ "l_linenumber": 1, "l_l_orderkey": 36 }
 { "l_linenumber": 1, "l_l_orderkey": 37 }
+{ "l_linenumber": 1, "l_l_orderkey": 38 }
+{ "l_linenumber": 1, "l_l_orderkey": 39 }
 { "l_linenumber": 1, "l_l_orderkey": 64 }
 { "l_linenumber": 1, "l_l_orderkey": 65 }
 { "l_linenumber": 1, "l_l_orderkey": 66 }
 { "l_linenumber": 1, "l_l_orderkey": 67 }
 { "l_linenumber": 1, "l_l_orderkey": 68 }
 { "l_linenumber": 1, "l_l_orderkey": 69 }
-{ "l_linenumber": 1, "l_l_orderkey": 97 }
-{ "l_linenumber": 1, "l_l_orderkey": 98 }
-{ "l_linenumber": 1, "l_l_orderkey": 5 }
-{ "l_linenumber": 1, "l_l_orderkey": 6 }
-{ "l_linenumber": 1, "l_l_orderkey": 36 }
-{ "l_linenumber": 1, "l_l_orderkey": 38 }
-{ "l_linenumber": 1, "l_l_orderkey": 39 }
 { "l_linenumber": 1, "l_l_orderkey": 70 }
-{ "l_linenumber": 1, "l_l_orderkey": 3 }
-{ "l_linenumber": 1, "l_l_orderkey": 7 }
-{ "l_linenumber": 1, "l_l_orderkey": 32 }
-{ "l_linenumber": 1, "l_l_orderkey": 33 }
-{ "l_linenumber": 1, "l_l_orderkey": 34 }
 { "l_linenumber": 1, "l_l_orderkey": 71 }
 { "l_linenumber": 1, "l_l_orderkey": 96 }
+{ "l_linenumber": 1, "l_l_orderkey": 97 }
+{ "l_linenumber": 1, "l_l_orderkey": 98 }
 { "l_linenumber": 1, "l_l_orderkey": 99 }
-{ "l_linenumber": 2, "l_l_orderkey": 32 }
-{ "l_linenumber": 2, "l_l_orderkey": 34 }
-{ "l_linenumber": 2, "l_l_orderkey": 39 }
-{ "l_linenumber": 2, "l_l_orderkey": 67 }
 { "l_linenumber": 2, "l_l_orderkey": 1 }
 { "l_linenumber": 2, "l_l_orderkey": 3 }
+{ "l_linenumber": 2, "l_l_orderkey": 5 }
 { "l_linenumber": 2, "l_l_orderkey": 7 }
+{ "l_linenumber": 2, "l_l_orderkey": 32 }
 { "l_linenumber": 2, "l_l_orderkey": 33 }
+{ "l_linenumber": 2, "l_l_orderkey": 34 }
 { "l_linenumber": 2, "l_l_orderkey": 35 }
-{ "l_linenumber": 2, "l_l_orderkey": 66 }
-{ "l_linenumber": 2, "l_l_orderkey": 5 }
-{ "l_linenumber": 2, "l_l_orderkey": 70 }
-{ "l_linenumber": 2, "l_l_orderkey": 71 }
-{ "l_linenumber": 2, "l_l_orderkey": 96 }
-{ "l_linenumber": 2, "l_l_orderkey": 99 }
 { "l_linenumber": 2, "l_l_orderkey": 37 }
+{ "l_linenumber": 2, "l_l_orderkey": 39 }
 { "l_linenumber": 2, "l_l_orderkey": 65 }
+{ "l_linenumber": 2, "l_l_orderkey": 66 }
+{ "l_linenumber": 2, "l_l_orderkey": 67 }
 { "l_linenumber": 2, "l_l_orderkey": 68 }
 { "l_linenumber": 2, "l_l_orderkey": 69 }
+{ "l_linenumber": 2, "l_l_orderkey": 70 }
+{ "l_linenumber": 2, "l_l_orderkey": 71 }
+{ "l_linenumber": 2, "l_l_orderkey": 96 }
 { "l_linenumber": 2, "l_l_orderkey": 97 }
 { "l_linenumber": 2, "l_l_orderkey": 98 }
+{ "l_linenumber": 2, "l_l_orderkey": 99 }
 { "l_linenumber": 3, "l_l_orderkey": 1 }
 { "l_linenumber": 3, "l_l_orderkey": 3 }
+{ "l_linenumber": 3, "l_l_orderkey": 5 }
 { "l_linenumber": 3, "l_l_orderkey": 7 }
-{ "l_linenumber": 3, "l_l_orderkey": 33 }
-{ "l_linenumber": 3, "l_l_orderkey": 35 }
-{ "l_linenumber": 3, "l_l_orderkey": 70 }
-{ "l_linenumber": 3, "l_l_orderkey": 71 }
-{ "l_linenumber": 3, "l_l_orderkey": 69 }
-{ "l_linenumber": 3, "l_l_orderkey": 98 }
 { "l_linenumber": 3, "l_l_orderkey": 32 }
+{ "l_linenumber": 3, "l_l_orderkey": 33 }
 { "l_linenumber": 3, "l_l_orderkey": 34 }
+{ "l_linenumber": 3, "l_l_orderkey": 35 }
 { "l_linenumber": 3, "l_l_orderkey": 37 }
 { "l_linenumber": 3, "l_l_orderkey": 39 }
 { "l_linenumber": 3, "l_l_orderkey": 65 }
 { "l_linenumber": 3, "l_l_orderkey": 67 }
 { "l_linenumber": 3, "l_l_orderkey": 68 }
+{ "l_linenumber": 3, "l_l_orderkey": 69 }
+{ "l_linenumber": 3, "l_l_orderkey": 70 }
+{ "l_linenumber": 3, "l_l_orderkey": 71 }
 { "l_linenumber": 3, "l_l_orderkey": 97 }
-{ "l_linenumber": 3, "l_l_orderkey": 5 }
+{ "l_linenumber": 3, "l_l_orderkey": 98 }
 { "l_linenumber": 3, "l_l_orderkey": 99 }
 { "l_linenumber": 4, "l_l_orderkey": 1 }
-{ "l_linenumber": 4, "l_l_orderkey": 35 }
-{ "l_linenumber": 4, "l_l_orderkey": 67 }
-{ "l_linenumber": 4, "l_l_orderkey": 68 }
-{ "l_linenumber": 4, "l_l_orderkey": 71 }
 { "l_linenumber": 4, "l_l_orderkey": 3 }
 { "l_linenumber": 4, "l_l_orderkey": 7 }
 { "l_linenumber": 4, "l_l_orderkey": 32 }
 { "l_linenumber": 4, "l_l_orderkey": 33 }
+{ "l_linenumber": 4, "l_l_orderkey": 35 }
 { "l_linenumber": 4, "l_l_orderkey": 39 }
-{ "l_linenumber": 4, "l_l_orderkey": 70 }
+{ "l_linenumber": 4, "l_l_orderkey": 67 }
+{ "l_linenumber": 4, "l_l_orderkey": 68 }
 { "l_linenumber": 4, "l_l_orderkey": 69 }
+{ "l_linenumber": 4, "l_l_orderkey": 70 }
+{ "l_linenumber": 4, "l_l_orderkey": 71 }
 { "l_linenumber": 4, "l_l_orderkey": 98 }
 { "l_linenumber": 4, "l_l_orderkey": 99 }
 { "l_linenumber": 5, "l_l_orderkey": 1 }
+{ "l_linenumber": 5, "l_l_orderkey": 3 }
 { "l_linenumber": 5, "l_l_orderkey": 7 }
-{ "l_linenumber": 5, "l_l_orderkey": 35 }
-{ "l_linenumber": 5, "l_l_orderkey": 70 }
-{ "l_linenumber": 5, "l_l_orderkey": 71 }
-{ "l_linenumber": 5, "l_l_orderkey": 69 }
 { "l_linenumber": 5, "l_l_orderkey": 32 }
+{ "l_linenumber": 5, "l_l_orderkey": 35 }
 { "l_linenumber": 5, "l_l_orderkey": 39 }
 { "l_linenumber": 5, "l_l_orderkey": 67 }
 { "l_linenumber": 5, "l_l_orderkey": 68 }
-{ "l_linenumber": 5, "l_l_orderkey": 3 }
-{ "l_linenumber": 6, "l_l_orderkey": 3 }
+{ "l_linenumber": 5, "l_l_orderkey": 69 }
+{ "l_linenumber": 5, "l_l_orderkey": 70 }
+{ "l_linenumber": 5, "l_l_orderkey": 71 }
 { "l_linenumber": 6, "l_l_orderkey": 1 }
+{ "l_linenumber": 6, "l_l_orderkey": 3 }
+{ "l_linenumber": 6, "l_l_orderkey": 7 }
+{ "l_linenumber": 6, "l_l_orderkey": 32 }
 { "l_linenumber": 6, "l_l_orderkey": 35 }
+{ "l_linenumber": 6, "l_l_orderkey": 39 }
 { "l_linenumber": 6, "l_l_orderkey": 67 }
 { "l_linenumber": 6, "l_l_orderkey": 68 }
-{ "l_linenumber": 6, "l_l_orderkey": 71 }
 { "l_linenumber": 6, "l_l_orderkey": 69 }
-{ "l_linenumber": 6, "l_l_orderkey": 7 }
-{ "l_linenumber": 6, "l_l_orderkey": 32 }
-{ "l_linenumber": 6, "l_l_orderkey": 39 }
 { "l_linenumber": 6, "l_l_orderkey": 70 }
-{ "l_linenumber": 7, "l_l_orderkey": 68 }
+{ "l_linenumber": 6, "l_l_orderkey": 71 }
 { "l_linenumber": 7, "l_l_orderkey": 7 }
+{ "l_linenumber": 7, "l_l_orderkey": 68 }
\ No newline at end of file


[09/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..b430256
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_correlated_ps.plan
@@ -0,0 +1,84 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$49(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$49(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[$$43]  |PARTITIONED|
+                                    {
+                                      -- AGGREGATE  |LOCAL|
+                                        -- STREAM_SELECT  |LOCAL|
+                                          -- UNNEST  |LOCAL|
+                                            -- MICRO_PRE_CLUSTERED_GROUP_BY[]  |LOCAL|
+                                                    {
+                                                      -- AGGREGATE  |LOCAL|
+                                                        -- STREAM_SELECT  |LOCAL|
+                                                          -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                    }
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                    }
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- HYBRID_HASH_JOIN [$$43][$$42]  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STREAM_SELECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                  -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ASSIGN  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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[$$43]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- STREAM_SELECT  |LOCAL|
+                                                  -- UNNEST  |LOCAL|
+                                                    -- MICRO_PRE_CLUSTERED_GROUP_BY[]  |LOCAL|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- STREAM_SELECT  |LOCAL|
+                                                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- HYBRID_HASH_JOIN [$$43][$$42]  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STREAM_SELECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- HASH_PARTITION_EXCHANGE [$$42]  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..74bd5b4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/in_ps.plan
@@ -0,0 +1,74 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$44(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$44(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[$$39]  |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 [$$39][$$29]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- STREAM_SELECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                -- 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[$$39]  |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 [$$39][$$29]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- STREAM_SELECT  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- DATASOURCE_SCAN  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$29]  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..5a3e2f6
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/subquery/not_exists_ps.plan
@@ -0,0 +1,157 @@
+-- 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[$$165]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$165]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$142]  |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[$$162]  |PARTITIONED|
+                                                  {
+                                                    -- AGGREGATE  |LOCAL|
+                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                  }
+                                            -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+                                              -- PRE_CLUSTERED_GROUP_BY[$$156]  |PARTITIONED|
+                                                      {
+                                                        -- AGGREGATE  |LOCAL|
+                                                          -- STREAM_SELECT  |LOCAL|
+                                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                      }
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- HYBRID_HASH_JOIN [$$151][$$148]  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                              -- ASSIGN  |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  |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  |PARTITIONED|
+                                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                            -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |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[$$165]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$165]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$142]  |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[$$162]  |PARTITIONED|
+                                                            {
+                                                              -- AGGREGATE  |LOCAL|
+                                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                            }
+                                                      -- HASH_PARTITION_EXCHANGE [$$162]  |PARTITIONED|
+                                                        -- PRE_CLUSTERED_GROUP_BY[$$156]  |PARTITIONED|
+                                                                {
+                                                                  -- AGGREGATE  |LOCAL|
+                                                                    -- STREAM_SELECT  |LOCAL|
+                                                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                                                }
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- STABLE_SORT [$$156(ASC)]  |PARTITIONED|
+                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                    -- HYBRID_HASH_JOIN [$$151][$$148]  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$151]  |PARTITIONED|
+                                                                        -- ASSIGN  |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  |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  |PARTITIONED|
+                                                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                                      -- HASH_PARTITION_EXCHANGE [$$148]  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..9a0cc83
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpcds/query-ASTERIXDB-1596_ps.plan
@@ -0,0 +1,58 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- STREAM_PROJECT  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STABLE_SORT [$$44(ASC), $$45(ASC)]  |PARTITIONED|
+              -- RANGE_PARTITION_EXCHANGE [$$44(ASC), $$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 [$$44][$$48]  |PARTITIONED|
+                              -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- HYBRID_HASH_JOIN [$$45, $$44][$$47, $$46]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- DATASOURCE_SCAN  |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][$$48]  |PARTITIONED|
+                                        -- HASH_PARTITION_EXCHANGE [$$44]  |PARTITIONED|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- HYBRID_HASH_JOIN [$$45, $$44][$$47, $$46]  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- DATASOURCE_SCAN  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..17aa91c
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_broadcast_ps.plan
@@ -0,0 +1,95 @@
+-- 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[$$122]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$112][$$108]  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |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  |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[$$122]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$112][$$108]  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |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  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..b7864dd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/tpch/q12_shipping_ps.plan
@@ -0,0 +1,95 @@
+-- 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[$$122]  |PARTITIONED|
+                              {
+                                -- AGGREGATE  |LOCAL|
+                                  -- NESTED_TUPLE_SOURCE  |LOCAL|
+                              }
+                        -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                          -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                  {
+                                    -- AGGREGATE  |LOCAL|
+                                      -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                  }
+                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                              -- STREAM_PROJECT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- HYBRID_HASH_JOIN [$$113][$$108]  |PARTITIONED|
+                                            -- HASH_PARTITION_EXCHANGE [$$113]  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- STREAM_SELECT  |PARTITIONED|
+                                                  -- ASSIGN  |PARTITIONED|
+                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- DATASOURCE_SCAN  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |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[$$122]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- HASH_PARTITION_EXCHANGE [$$122]  |PARTITIONED|
+                                    -- SORT_GROUP_BY[$$104]  |PARTITIONED|
+                                            {
+                                              -- AGGREGATE  |LOCAL|
+                                                -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                            }
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- HYBRID_HASH_JOIN [$$104][$$111]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- HYBRID_HASH_JOIN [$$113][$$108]  |PARTITIONED|
+                                                      -- HASH_PARTITION_EXCHANGE [$$113]  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- STREAM_SELECT  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                -- UNNEST  |UNPARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql
index 59a590e..42c2097 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/big-object/big_object_insert/big_object_insert.3.query.aql
@@ -18,6 +18,8 @@
  */
 use dataverse testdv2;
 
+set "compiler.sortmemory" "32MB"
+
 for $d in dataset("testds")
-order by $d.id
+order by $d.name desc, $d.id
 return $d

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql
index 77b930e..f0e735e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/prefix-search/prefix-search.3.query.aql
@@ -28,7 +28,7 @@ use dataverse tpch;
 for $l in dataset('LineItem')
 where $l.l_orderkey>=0
       and $l.l_orderkey<100
-order by $l.l_linenumber
+order by $l.l_linenumber, $l.l_orderkey
 return {
   "l_linenumber": $l.l_linenumber,
   "l_l_orderkey": $l.l_orderkey

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql
index a412072..a3ccb68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/misc/stable_sort/stable_sort.3.query.aql
@@ -25,5 +25,5 @@
 use dataverse test;
 
 for $i in dataset LineItem
-order by $i.l_partkey desc
+order by $i.l_partkey desc, $i.l_orderkey, $i.l_linenumber
 return $i

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql
index 4bb5ca9..1c7faa6 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.aql
@@ -26,9 +26,9 @@ use dataverse test;
 for $i in dataset tsdata
   order by $i.time
   for $j in overlap-bins(interval-start-from-time($i.time, $i.duration), time("00:00:00"), day-time-duration("PT1H30M"))
-    group by $bin := $j with $i 
+    group by $bin := $j with $i
     order by get-interval-start($bin)
     for $x in $i
       let $itv := interval-start-from-time($x.time, $x.duration)
-      order by get-interval-start($bin)
+      order by get-interval-start($bin), $itv, get-overlapping-interval($bin, $itv)
       return { "tbin": $bin, "interval": $itv, "overlap": get-overlapping-interval($bin, $itv) }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.1.ddl.sqlpp
new file mode 100644
index 0000000..d29e181
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.1.ddl.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.
+ */
+/*
+ * Description: testing setting the number of samples for parallel sort
+ */
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+
+use tpch;
+
+create type tpch.LineItemType as closed {
+  l_orderkey : integer,
+  l_partkey : integer,
+  l_suppkey : integer,
+  l_linenumber : integer,
+  l_quantity : double,
+  l_extendedprice : double,
+  l_discount : double,
+  l_tax : double,
+  l_returnflag : string,
+  l_linestatus : string,
+  l_shipdate : string,
+  l_commitdate : string,
+  l_receiptdate : string,
+  l_shipinstruct : string,
+  l_shipmode : string,
+  l_comment : string
+};
+
+create  dataset LineItem(LineItemType) primary key l_orderkey,l_linenumber;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.2.update.sqlpp
new file mode 100644
index 0000000..8a78296
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.2.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 tpch;
+
+load  dataset LineItem using localfs ((`path`=`asterix_nc1://data/tpch0.001/lineitem.tbl`),(`format`=`delimited-text`),(`delimiter`=`|`)) pre-sorted;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.3.query.sqlpp
new file mode 100644
index 0000000..612b37a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_num_samples/p_sort_num_samples.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 tpch;
+
+SET `compiler.sort.samples` "400";
+
+SELECT VALUE v
+FROM LineItem v
+ORDER BY v.l_partkey, v.l_orderkey, v.l_linenumber;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.1.ddl.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.1.ddl.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.1.ddl.sqlpp
new file mode 100644
index 0000000..af54590
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.1.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.
+ */
+
+/*
+ * Description: testing a sequential merge when parallel sort has redistributed the data across partitions and one of
+ * the next operators requires merging the sorted data.
+ */
+
+drop dataverse test if exists;
+create dataverse test;
+
+use test;
+
+create type TestType as
+{
+  id: int,
+  f1: int
+};
+
+create  dataset TestDS(TestType) primary key id;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.2.update.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.2.update.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.2.update.sqlpp
new file mode 100644
index 0000000..5aacecf
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.2.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;
+
+INSERT INTO TestDS
+([
+{"id":1, "f1":40},
+{"id":2, "f1":40},
+{"id":3, "f1":40},
+{"id":4, "f1":41},
+{"id":5, "f1":42},
+{"id":6, "f1":42},
+{"id":7, "f1":40},
+{"id":8, "f1":41},
+{"id":9, "f1":41},
+{"id":10, "f1":42},
+{"id":11, "f1":40},
+{"id":12, "f1":41},
+{"id":13, "f1":42},
+{"id":14, "f1":41},
+{"id":15, "f1":40},
+{"id":16, "f1":42},
+{"id":17, "f1":42},
+{"id":18, "f1":41}
+]);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.3.query.sqlpp
new file mode 100644
index 0000000..9c38f40
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/p_sort_seq_merge/p_sort_seq_merge.3.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;
+
+set `compiler.sort.parallel` "true";
+
+[(select * from TestDS  v order by v.f1, v.id)];

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp
index 6837e2d..a397143 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/prefix-search/prefix-search.3.query.sqlpp
@@ -29,5 +29,5 @@ use tpch;
 select element {'l_linenumber':l.l_linenumber,'l_l_orderkey':l.l_orderkey}
 from  LineItem as l
 where ((l.l_orderkey >= 0) and (l.l_orderkey < 100))
-order by l.l_linenumber
+order by l.l_linenumber, l.l_orderkey
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp
index dbcb4f0..baa27af 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/misc/stable_sort/stable_sort.3.query.sqlpp
@@ -24,8 +24,7 @@
 
 use test;
 
-
 select element i
 from  LineItem as i
-order by i.l_partkey desc
+order by i.l_partkey desc, i.l_orderkey, i.l_linenumber
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
index f7eabc3..f6f4a5c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/objects/object_remove/object_remove.3.query.sqlpp
@@ -39,7 +39,7 @@ use TinySocial;
   "t5": (
     select value object_remove(u, "lang")
     from TwitterUsers as u
-    order by u.screen-name
+    order by u.`screen-name`
   ),
 
   /* closed type */

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp
index d8fda37..ee9b52d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.13.query.sqlpp
@@ -19,6 +19,8 @@
 
 use tpch;
 
+set `compiler.sort.parallel` "false";
+
 explain select value l
 from LineItem l
 where l_shipdate="1994-01-20"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp
index d8fda37..ee9b52d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/rebalance/single_dataset_with_index/single_dataset_with_index.8.query.sqlpp
@@ -19,6 +19,8 @@
 
 use tpch;
 
+set `compiler.sort.parallel` "false";
+
 explain select value l
 from LineItem l
 where l_shipdate="1994-01-20"

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp
index 07fa4fa..70a0e7e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/temporal/overlap_bins_gby_0/overlap_bins_gby_0.3.query.sqlpp
@@ -35,4 +35,4 @@ from
     ) as gen0,
     gen0.i as x
 let itv = test.`interval-start-from-time`(x.time,x.duration)
-order by test.`get-interval-start`(gen0.bin);
+order by test.`get-interval-start`(gen0.bin), itv, test.`get-overlapping-interval`(gen0.bin,itv);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.regexadm
----------------------------------------------------------------------
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 79db9ae..050a799 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
@@ -12,6 +12,8 @@
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
     "compiler\.parallelism" : 0,
+    "compiler\.sort\.parallel" : false,
+    "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.regexadm
----------------------------------------------------------------------
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 6e5547d..c56062a 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
@@ -12,6 +12,8 @@
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
     "compiler\.parallelism" : -1,
+    "compiler\.sort\.parallel" : true,
+    "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.regexadm
----------------------------------------------------------------------
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 3237551..3a3796d 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
@@ -12,6 +12,8 @@
     "compiler\.groupmemory" : 163840,
     "compiler\.joinmemory" : 262144,
     "compiler\.parallelism" : 3,
+    "compiler\.sort\.parallel" : true,
+    "compiler\.sort\.samples" : 100,
     "compiler\.sortmemory" : 327680,
     "compiler\.textsearchmemory" : 163840,
     "default\.dir" : "target/io/dir/asterixdb",


[08/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_num_samples/p_sort_num_samples.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_num_samples/p_sort_num_samples.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_num_samples/p_sort_num_samples.3.adm
new file mode 100644
index 0000000..e43af35
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/misc/p_sort_num_samples/p_sort_num_samples.3.adm
@@ -0,0 +1,6005 @@
+{ "l_orderkey": 35, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 21624.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-21", "l_commitdate": "1996-01-03", "l_receiptdate": "1996-03-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": ", regular tithe" }
+{ "l_orderkey": 134, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-07-17", "l_commitdate": "1992-07-08", "l_receiptdate": "1992-07-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "s. quickly regular" }
+{ "l_orderkey": 321, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.01, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-18", "l_commitdate": "1993-04-24", "l_receiptdate": "1993-08-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "hockey players sleep slyly sl" }
+{ "l_orderkey": 548, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18921.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-13", "l_commitdate": "1994-12-18", "l_receiptdate": "1995-01-25", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ideas. special accounts above the furiou" }
+{ "l_orderkey": 640, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 40.0, "l_extendedprice": 36040.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-11", "l_commitdate": "1993-04-11", "l_receiptdate": "1993-05-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "oach according to the bol" }
+{ "l_orderkey": 807, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 19.0, "l_extendedprice": 17119.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-10", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ns haggle quickly across the furi" }
+{ "l_orderkey": 1122, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 34238.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-23", "l_commitdate": "1997-04-02", "l_receiptdate": "1997-02-16", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "t theodolites sleep. even, ironic" }
+{ "l_orderkey": 1154, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 35.0, "l_extendedprice": 31535.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-03-30", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-04-21", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "the carefully regular pinto beans boost" }
+{ "l_orderkey": 1287, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-12", "l_commitdate": "1994-09-23", "l_receiptdate": "1994-08-07", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ar packages. even, even" }
+{ "l_orderkey": 1472, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-24", "l_commitdate": "1996-11-19", "l_receiptdate": "1996-11-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "onic theodolites hinder slyly slyly r" }
+{ "l_orderkey": 1668, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 25.0, "l_extendedprice": 22525.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-08-08", "l_commitdate": "1997-09-28", "l_receiptdate": "1997-09-01", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "y ironic requests. bold, final ideas a" }
+{ "l_orderkey": 1761, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 7, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-06", "l_commitdate": "1994-03-18", "l_receiptdate": "1994-03-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ons boost fu" }
+{ "l_orderkey": 2117, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 6, "l_quantity": 27.0, "l_extendedprice": 24327.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-30", "l_commitdate": "1997-06-27", "l_receiptdate": "1997-07-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " the carefully ironic ideas" }
+{ "l_orderkey": 2374, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22525.0, "l_discount": 0.08, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-26", "l_commitdate": "1993-12-15", "l_receiptdate": "1993-12-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "refully pending d" }
+{ "l_orderkey": 2528, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 10.0, "l_extendedprice": 9010.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-12-12", "l_commitdate": "1994-12-29", "l_receiptdate": "1994-12-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ely. fluffily even re" }
+{ "l_orderkey": 2534, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 50.0, "l_extendedprice": 45050.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-25", "l_commitdate": "1996-10-07", "l_receiptdate": "1996-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ideas. deposits use. slyly regular pa" }
+{ "l_orderkey": 2726, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 45050.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-03-04", "l_commitdate": "1993-01-29", "l_receiptdate": "1993-03-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " furiously bold theodolites" }
+{ "l_orderkey": 2883, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 29733.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-02-26", "l_commitdate": "1995-03-04", "l_receiptdate": "1995-03-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "s. final i" }
+{ "l_orderkey": 2885, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 45.0, "l_extendedprice": 40545.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-12-24", "l_commitdate": "1992-10-30", "l_receiptdate": "1993-01-04", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ess ideas. regular, silen" }
+{ "l_orderkey": 3175, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34238.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-10", "l_commitdate": "1994-08-25", "l_receiptdate": "1994-10-28", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "the quickly even dolph" }
+{ "l_orderkey": 3175, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 28832.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-29", "l_commitdate": "1994-09-20", "l_receiptdate": "1994-10-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "lites sleep" }
+{ "l_orderkey": 3457, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 21624.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-03", "l_commitdate": "1995-05-30", "l_receiptdate": "1995-08-14", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "tructions haggle alongsid" }
+{ "l_orderkey": 3843, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 30.0, "l_extendedprice": 27030.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-14", "l_commitdate": "1997-03-25", "l_receiptdate": "1997-03-13", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " wake. slyly even packages boost " }
+{ "l_orderkey": 3940, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 36941.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-08", "l_commitdate": "1996-05-03", "l_receiptdate": "1996-06-03", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "thily. deposits cajole." }
+{ "l_orderkey": 4102, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 32.0, "l_extendedprice": 28832.0, "l_discount": 0.08, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-14", "l_commitdate": "1996-04-29", "l_receiptdate": "1996-05-29", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " the even requests; regular pinto" }
+{ "l_orderkey": 4293, "l_partkey": 1, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 30634.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-05", "l_commitdate": "1996-10-12", "l_receiptdate": "1996-12-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "ions sleep blithely on" }
+{ "l_orderkey": 4323, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 33.0, "l_extendedprice": 29733.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-04", "l_commitdate": "1994-03-06", "l_receiptdate": "1994-05-23", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "the slyly bold deposits slee" }
+{ "l_orderkey": 4355, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 13.0, "l_extendedprice": 11713.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-21", "l_commitdate": "1996-12-22", "l_receiptdate": "1997-02-14", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": " ought to mold. blithely pending ideas " }
+{ "l_orderkey": 4452, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42347.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-08", "l_commitdate": "1994-08-09", "l_receiptdate": "1994-10-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ts. slyly regular cour" }
+{ "l_orderkey": 4580, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 41.0, "l_extendedprice": 36941.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-13", "l_commitdate": "1994-01-31", "l_receiptdate": "1994-01-06", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "requests. quickly silent asymptotes sle" }
+{ "l_orderkey": 5121, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 6, "l_quantity": 2.0, "l_extendedprice": 1802.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-08-10", "l_commitdate": "1992-06-28", "l_receiptdate": "1992-08-11", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " final, regular account" }
+{ "l_orderkey": 5409, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8109.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-02-15", "l_commitdate": "1992-04-02", "l_receiptdate": "1992-02-28", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " unusual, unusual reques" }
+{ "l_orderkey": 5634, "l_partkey": 1, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 1.0, "l_extendedprice": 901.0, "l_discount": 0.04, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-02", "l_commitdate": "1996-10-21", "l_receiptdate": "1996-10-27", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ctions haggle carefully. carefully clo" }
+{ "l_orderkey": 5760, "l_partkey": 1, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5406.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-30", "l_commitdate": "1994-07-31", "l_receiptdate": "1994-08-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ng the acco" }
+{ "l_orderkey": 5984, "l_partkey": 1, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 8.0, "l_extendedprice": 7208.0, "l_discount": 0.1, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-09-17", "l_commitdate": "1994-08-28", "l_receiptdate": "1994-09-25", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its. express," }
+{ "l_orderkey": 65, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18942.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-06", "l_commitdate": "1995-05-14", "l_receiptdate": "1995-07-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "bove the even packages. accounts nag carefu" }
+{ "l_orderkey": 130, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 48.0, "l_extendedprice": 43296.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-01", "l_commitdate": "1992-07-12", "l_receiptdate": "1992-07-24", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lithely alongside of the regu" }
+{ "l_orderkey": 261, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 34.0, "l_extendedprice": 30668.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-18", "l_commitdate": "1993-09-24", "l_receiptdate": "1993-08-20", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "c packages. asymptotes da" }
+{ "l_orderkey": 290, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 5.0, "l_extendedprice": 4510.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-19", "l_commitdate": "1994-02-24", "l_receiptdate": "1994-01-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ans integrate. requests sleep. fur" }
+{ "l_orderkey": 418, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-23", "l_commitdate": "1995-06-16", "l_receiptdate": "1995-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "regular, silent pinto" }
+{ "l_orderkey": 740, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-08-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "odolites cajole ironic, pending instruc" }
+{ "l_orderkey": 896, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 7.0, "l_extendedprice": 6314.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-02", "l_commitdate": "1993-05-24", "l_receiptdate": "1993-05-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " requests " }
+{ "l_orderkey": 1250, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-05", "l_commitdate": "1992-12-17", "l_receiptdate": "1992-12-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " regular, i" }
+{ "l_orderkey": 1575, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10824.0, "l_discount": 0.01, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-27", "l_commitdate": "1995-11-11", "l_receiptdate": "1996-01-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": " bold accounts. furi" }
+{ "l_orderkey": 1696, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17138.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-03", "l_commitdate": "1998-03-13", "l_receiptdate": "1998-05-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "its maintain alongside of the f" }
+{ "l_orderkey": 2370, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21648.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-04-09", "l_receiptdate": "1994-06-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "final depen" }
+{ "l_orderkey": 2437, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 23.0, "l_extendedprice": 20746.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-15", "l_commitdate": "1993-06-28", "l_receiptdate": "1993-08-23", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "s deposits. pendi" }
+{ "l_orderkey": 2656, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17138.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-03", "l_commitdate": "1993-07-25", "l_receiptdate": "1993-08-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ts serve deposi" }
+{ "l_orderkey": 2662, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5412.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-30", "l_commitdate": "1996-09-20", "l_receiptdate": "1996-12-03", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "olites cajole quickly along the b" }
+{ "l_orderkey": 3046, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 27962.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-24", "l_commitdate": "1996-01-30", "l_receiptdate": "1996-03-26", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "y pending somas alongside of the slyly iro" }
+{ "l_orderkey": 3362, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 3.0, "l_extendedprice": 2706.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-26", "l_commitdate": "1995-09-02", "l_receiptdate": "1995-09-17", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "its cajole blithely excuses. de" }
+{ "l_orderkey": 3650, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-07-18", "l_receiptdate": "1992-07-08", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "re about the pinto " }
+{ "l_orderkey": 3654, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-09-22", "l_commitdate": "1992-07-20", "l_receiptdate": "1992-10-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "unts doze bravely ab" }
+{ "l_orderkey": 3811, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 35.0, "l_extendedprice": 31570.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-17", "l_commitdate": "1998-06-30", "l_receiptdate": "1998-04-25", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "yly final dolphins? quickly ironic frets" }
+{ "l_orderkey": 4001, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35178.0, "l_discount": 0.0, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-13", "l_commitdate": "1997-06-17", "l_receiptdate": "1997-06-25", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": " dogged excuses. blithe" }
+{ "l_orderkey": 4032, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24354.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-31", "l_commitdate": "1998-04-19", "l_receiptdate": "1998-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "le furiously according to" }
+{ "l_orderkey": 4135, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 23.0, "l_extendedprice": 20746.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-09", "l_commitdate": "1997-05-12", "l_receiptdate": "1997-04-16", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "posits cajole furiously carefully" }
+{ "l_orderkey": 4387, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13530.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-11", "l_commitdate": "1996-01-14", "l_receiptdate": "1996-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": "s hinder quietly across the pla" }
+{ "l_orderkey": 4389, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.01, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-06-07", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "at the final excuses hinder carefully a" }
+{ "l_orderkey": 4454, "l_partkey": 2, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 1.0, "l_extendedprice": 902.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-05", "l_commitdate": "1994-04-19", "l_receiptdate": "1994-02-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "equests run." }
+{ "l_orderkey": 5090, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 22.0, "l_extendedprice": 19844.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-03", "l_commitdate": "1997-04-12", "l_receiptdate": "1997-07-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ular requests su" }
+{ "l_orderkey": 5312, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 38786.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-24", "l_commitdate": "1995-05-07", "l_receiptdate": "1995-03-28", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ly unusual" }
+{ "l_orderkey": 5348, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 37.0, "l_extendedprice": 33374.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-01", "l_commitdate": "1998-02-02", "l_receiptdate": "1997-12-07", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "y according to the carefully pending acco" }
+{ "l_orderkey": 5478, "l_partkey": 2, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 47.0, "l_extendedprice": 42394.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-08-15", "l_commitdate": "1996-07-12", "l_receiptdate": "1996-08-31", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " instructions; slyly even accounts hagg" }
+{ "l_orderkey": 5699, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 24.0, "l_extendedprice": 21648.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-10-21", "l_commitdate": "1992-09-04", "l_receiptdate": "1992-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "kages. fin" }
+{ "l_orderkey": 5828, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25256.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-20", "l_receiptdate": "1994-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " special ideas haggle slyly ac" }
+{ "l_orderkey": 5862, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26158.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-04-02", "l_commitdate": "1997-04-16", "l_receiptdate": "1997-04-04", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "e fluffily. furiously" }
+{ "l_orderkey": 5893, "l_partkey": 2, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1804.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-07-18", "l_commitdate": "1992-09-10", "l_receiptdate": "1992-08-12", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages wake sly" }
+{ "l_orderkey": 5957, "l_partkey": 2, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15334.0, "l_discount": 0.01, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-24", "l_commitdate": "1994-02-16", "l_receiptdate": "1994-02-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": ". final, pending packages" }
+{ "l_orderkey": 1, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-04-21", "l_commitdate": "1996-03-30", "l_receiptdate": "1996-05-16", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "lites. fluffily even de" }
+{ "l_orderkey": 32, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3612.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-04", "l_commitdate": "1995-10-01", "l_receiptdate": "1995-09-03", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "e slyly final pac" }
+{ "l_orderkey": 39, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 44.0, "l_extendedprice": 39732.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-14", "l_commitdate": "1996-12-15", "l_receiptdate": "1996-12-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "eodolites. careful" }
+{ "l_orderkey": 129, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41538.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-15", "l_commitdate": "1993-01-24", "l_receiptdate": "1993-03-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "uietly bold theodolites. fluffil" }
+{ "l_orderkey": 194, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 17.0, "l_extendedprice": 15351.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-24", "l_commitdate": "1992-05-22", "l_receiptdate": "1992-05-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": " regular deposi" }
+{ "l_orderkey": 519, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 38.0, "l_extendedprice": 34314.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-02-19", "l_commitdate": "1997-12-15", "l_receiptdate": "1998-03-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "gular excuses detect quickly furiously " }
+{ "l_orderkey": 801, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 21.0, "l_extendedprice": 18963.0, "l_discount": 0.05, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-04-25", "l_commitdate": "1992-03-20", "l_receiptdate": "1992-05-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cial, special packages." }
+{ "l_orderkey": 993, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-24", "l_commitdate": "1995-11-20", "l_receiptdate": "1995-11-06", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "lites. even theodolite" }
+{ "l_orderkey": 999, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 10.0, "l_extendedprice": 9030.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-11-23", "l_commitdate": "1993-12-02", "l_receiptdate": "1993-11-29", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "efully pending" }
+{ "l_orderkey": 1186, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.08, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-08", "l_commitdate": "1996-10-17", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "ffily spec" }
+{ "l_orderkey": 1508, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-22", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "cording to the furiously ironic depe" }
+{ "l_orderkey": 1542, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 12.0, "l_extendedprice": 10836.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-29", "l_commitdate": "1993-11-02", "l_receiptdate": "1993-11-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "carefully " }
+{ "l_orderkey": 2372, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 17.0, "l_extendedprice": 15351.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-17", "l_commitdate": "1998-01-17", "l_receiptdate": "1997-12-25", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "xcuses. slyly ironic theod" }
+{ "l_orderkey": 2401, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 49.0, "l_extendedprice": 44247.0, "l_discount": 0.05, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-02", "l_commitdate": "1997-09-11", "l_receiptdate": "1997-09-13", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "lites cajole carefully " }
+{ "l_orderkey": 2721, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1806.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-13", "l_commitdate": "1996-03-14", "l_receiptdate": "1996-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " slyly final requests against " }
+{ "l_orderkey": 2946, "l_partkey": 3, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 35.0, "l_extendedprice": 31605.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-15", "l_commitdate": "1996-04-02", "l_receiptdate": "1996-03-26", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " sublate along the fluffily iron" }
+{ "l_orderkey": 2951, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.03, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-27", "l_commitdate": "1996-04-16", "l_receiptdate": "1996-03-30", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "to beans wake ac" }
+{ "l_orderkey": 3015, "l_partkey": 3, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4515.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-10", "l_commitdate": "1992-12-02", "l_receiptdate": "1993-01-19", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": " the furiously pendi" }
+{ "l_orderkey": 3110, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 34.0, "l_extendedprice": 30702.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-23", "l_commitdate": "1995-01-27", "l_receiptdate": "1995-03-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ly pending requests ha" }
+{ "l_orderkey": 3137, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5418.0, "l_discount": 0.02, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-19", "l_commitdate": "1995-10-23", "l_receiptdate": "1995-10-16", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ly express as" }
+{ "l_orderkey": 3331, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23478.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-05", "l_commitdate": "1993-07-17", "l_receiptdate": "1993-08-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "p asymptotes. carefully unusual in" }
+{ "l_orderkey": 3776, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 1, "l_quantity": 39.0, "l_extendedprice": 35217.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-03", "l_commitdate": "1993-02-05", "l_receiptdate": "1993-01-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "yly blithely pending packages" }
+{ "l_orderkey": 3937, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 5, "l_quantity": 29.0, "l_extendedprice": 26187.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-06", "l_commitdate": "1998-02-22", "l_receiptdate": "1998-03-14", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "nt pinto beans above the pending instr" }
+{ "l_orderkey": 4484, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 42.0, "l_extendedprice": 37926.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-25", "l_commitdate": "1997-02-21", "l_receiptdate": "1997-04-05", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ding, pending requests wake. fluffily " }
+{ "l_orderkey": 4740, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 22.0, "l_extendedprice": 19866.0, "l_discount": 0.06, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-04", "l_commitdate": "1996-08-17", "l_receiptdate": "1996-10-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "final dependencies nag " }
+{ "l_orderkey": 5252, "l_partkey": 3, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37023.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-03-16", "l_commitdate": "1996-04-18", "l_receiptdate": "1996-03-17", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ording to the blithely express somas sho" }
+{ "l_orderkey": 5892, "l_partkey": 3, "l_suppkey": 4, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 25284.0, "l_discount": 0.03, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-08-16", "l_commitdate": "1995-07-06", "l_receiptdate": "1995-08-22", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ithely unusual accounts will have to integ" }
+{ "l_orderkey": 164, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 23.0, "l_extendedprice": 20792.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-11-03", "l_commitdate": "1992-12-02", "l_receiptdate": "1992-11-12", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "ress packages haggle ideas. blithely spec" }
+{ "l_orderkey": 641, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37064.0, "l_discount": 0.07, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-11-29", "l_commitdate": "1993-10-27", "l_receiptdate": "1993-12-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": " asymptotes are quickly. bol" }
+{ "l_orderkey": 704, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12656.0, "l_discount": 0.07, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-02-02", "l_commitdate": "1996-12-26", "l_receiptdate": "1997-02-19", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ve the quickly final forges. furiously p" }
+{ "l_orderkey": 739, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 50.0, "l_extendedprice": 45200.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-26", "l_commitdate": "1998-07-16", "l_receiptdate": "1998-09-02", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ndencies. blith" }
+{ "l_orderkey": 1251, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 37.0, "l_extendedprice": 33448.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-21", "l_commitdate": "1998-01-12", "l_receiptdate": "1997-12-26", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": ". furiously" }
+{ "l_orderkey": 1316, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 6, "l_quantity": 7.0, "l_extendedprice": 6328.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-09", "l_commitdate": "1994-01-12", "l_receiptdate": "1993-12-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ". furiously even accounts a" }
+{ "l_orderkey": 2019, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 31.0, "l_extendedprice": 28024.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-18", "l_commitdate": "1992-12-26", "l_receiptdate": "1992-11-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "l ideas across the slowl" }
+{ "l_orderkey": 2144, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.0, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-03", "l_commitdate": "1994-05-16", "l_receiptdate": "1994-06-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "ns wake carefully carefully ironic" }
+{ "l_orderkey": 2279, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-31", "l_commitdate": "1993-05-07", "l_receiptdate": "1993-06-05", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ing foxes above the even accounts use slyly" }
+{ "l_orderkey": 2404, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 18.0, "l_extendedprice": 16272.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-25", "l_commitdate": "1997-05-06", "l_receiptdate": "1997-07-02", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "packages. even requests according to " }
+{ "l_orderkey": 2560, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 27.0, "l_extendedprice": 24408.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-12-03", "l_commitdate": "1992-11-16", "l_receiptdate": "1992-12-30", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " against the carefully" }
+{ "l_orderkey": 2593, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-12-16", "l_commitdate": "1993-11-01", "l_receiptdate": "1993-12-29", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "the furiously " }
+{ "l_orderkey": 2790, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 32.0, "l_extendedprice": 28928.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-25", "l_commitdate": "1994-10-26", "l_receiptdate": "1994-10-01", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "ully pending" }
+{ "l_orderkey": 2882, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12656.0, "l_discount": 0.09, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-28", "l_commitdate": "1995-11-11", "l_receiptdate": "1995-10-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "kly. even requests w" }
+{ "l_orderkey": 2885, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-05", "l_commitdate": "1992-12-12", "l_receiptdate": "1993-01-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "ctions solve. slyly regular requests n" }
+{ "l_orderkey": 2976, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21696.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-03-19", "l_commitdate": "1994-01-26", "l_receiptdate": "1994-04-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "ronic pinto beans. slyly bol" }
+{ "l_orderkey": 3522, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-21", "l_commitdate": "1994-12-09", "l_receiptdate": "1995-01-23", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "tes snooze " }
+{ "l_orderkey": 4005, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 26.0, "l_extendedprice": 23504.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-01", "l_commitdate": "1997-02-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " to the quic" }
+{ "l_orderkey": 4099, "l_partkey": 4, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 29.0, "l_extendedprice": 26216.0, "l_discount": 0.09, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-21", "l_commitdate": "1992-11-04", "l_receiptdate": "1992-11-30", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " slowly final warthogs sleep blithely. q" }
+{ "l_orderkey": 4196, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 7, "l_quantity": 3.0, "l_extendedprice": 2712.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-08-05", "l_commitdate": "1998-07-28", "l_receiptdate": "1998-08-15", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "y regular packages haggle furiously alongs" }
+{ "l_orderkey": 4292, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 47.0, "l_extendedprice": 42488.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-03-21", "l_receiptdate": "1992-05-27", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "y packages; even ideas boost" }
+{ "l_orderkey": 4736, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 43.0, "l_extendedprice": 38872.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-05", "l_commitdate": "1995-12-21", "l_receiptdate": "1996-02-06", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "quests. carefully " }
+{ "l_orderkey": 5349, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 6.0, "l_extendedprice": 5424.0, "l_discount": 0.1, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-30", "l_commitdate": "1996-10-08", "l_receiptdate": "1997-01-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "inal deposits affix carefully" }
+{ "l_orderkey": 5411, "l_partkey": 4, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 19.0, "l_extendedprice": 17176.0, "l_discount": 0.05, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-05-25", "l_commitdate": "1997-07-30", "l_receiptdate": "1997-06-19", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "ial accounts according to the f" }
+{ "l_orderkey": 5668, "l_partkey": 4, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 15.0, "l_extendedprice": 13560.0, "l_discount": 0.03, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-04-06", "l_commitdate": "1995-05-12", "l_receiptdate": "1995-04-17", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": " the express, pending requests. bo" }
+{ "l_orderkey": 5856, "l_partkey": 4, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 1.0, "l_extendedprice": 904.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-12-29", "l_commitdate": "1995-01-07", "l_receiptdate": "1995-01-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "tly. special deposits wake blithely even" }
+{ "l_orderkey": 3, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40725.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-02", "l_commitdate": "1994-01-04", "l_receiptdate": "1994-02-23", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ongside of the furiously brave acco" }
+{ "l_orderkey": 228, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2715.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-20", "l_commitdate": "1993-04-08", "l_receiptdate": "1993-05-26", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ckages. sly" }
+{ "l_orderkey": 320, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 30.0, "l_extendedprice": 27150.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-04", "l_commitdate": "1998-01-21", "l_receiptdate": "1997-12-13", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": " ironic, final accounts wake quick de" }
+{ "l_orderkey": 354, "l_partkey": 5, "l_suppkey": 10, "l_linenumber": 7, "l_quantity": 14.0, "l_extendedprice": 12670.0, "l_discount": 0.01, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-06", "l_commitdate": "1996-06-08", "l_receiptdate": "1996-07-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "t thinly above the ironic, " }
+{ "l_orderkey": 548, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 6.0, "l_extendedprice": 5430.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-01-18", "l_commitdate": "1994-12-08", "l_receiptdate": "1995-02-10", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "sits wake furiously regular" }
+{ "l_orderkey": 645, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 43.0, "l_extendedprice": 38915.0, "l_discount": 0.06, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-12", "l_commitdate": "1995-02-27", "l_receiptdate": "1995-03-06", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "REG AIR", "l_comment": " furiously accounts. slyly" }
+{ "l_orderkey": 675, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 46.0, "l_extendedprice": 41630.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-09-18", "l_commitdate": "1997-10-14", "l_receiptdate": "1997-10-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": " deposits along the express foxes " }
+{ "l_orderkey": 966, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 20.0, "l_extendedprice": 18100.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-19", "l_commitdate": "1998-07-15", "l_receiptdate": "1998-07-27", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "pecial ins" }
+{ "l_orderkey": 993, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 15.0, "l_extendedprice": 13575.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-09-27", "l_commitdate": "1995-10-21", "l_receiptdate": "1995-10-17", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "sits. pending pinto beans haggle? ca" }
+{ "l_orderkey": 1058, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 4, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.09, "l_tax": 0.01, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-05-27", "l_commitdate": "1993-06-10", "l_receiptdate": "1993-06-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": " the final requests believe carefully " }
+{ "l_orderkey": 1155, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 44345.0, "l_discount": 0.04, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-12-07", "l_commitdate": "1997-12-30", "l_receiptdate": "1997-12-08", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ccounts are alongside of t" }
+{ "l_orderkey": 1574, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 6, "l_quantity": 42.0, "l_extendedprice": 38010.0, "l_discount": 0.07, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-19", "l_commitdate": "1997-01-13", "l_receiptdate": "1996-12-28", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "o beans according t" }
+{ "l_orderkey": 1633, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13575.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-13", "l_commitdate": "1995-11-13", "l_receiptdate": "1996-01-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "ges wake fluffil" }
+{ "l_orderkey": 1637, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.05, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-06-07", "l_commitdate": "1995-03-26", "l_receiptdate": "1995-06-08", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " haggle carefully silent accou" }
+{ "l_orderkey": 1732, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 50.0, "l_extendedprice": 45250.0, "l_discount": 0.02, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-12-05", "l_commitdate": "1994-01-23", "l_receiptdate": "1993-12-20", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "fily final asymptotes according " }
+{ "l_orderkey": 1829, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9955.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-18", "l_commitdate": "1994-06-13", "l_receiptdate": "1994-06-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ding orbits" }
+{ "l_orderkey": 2178, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 40.0, "l_extendedprice": 36200.0, "l_discount": 0.0, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-17", "l_commitdate": "1997-02-09", "l_receiptdate": "1997-04-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "foxes are slowly regularly specia" }
+{ "l_orderkey": 2241, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-11", "l_commitdate": "1993-07-23", "l_receiptdate": "1993-09-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": " final deposits use fluffily. even f" }
+{ "l_orderkey": 2375, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 4, "l_quantity": 5.0, "l_extendedprice": 4525.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-01-31", "l_commitdate": "1997-01-25", "l_receiptdate": "1997-02-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "final packages cajole according to the furi" }
+{ "l_orderkey": 2725, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.01, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-07-05", "l_commitdate": "1994-06-29", "l_receiptdate": "1994-08-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "ns sleep furiously c" }
+{ "l_orderkey": 2819, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 3, "l_quantity": 28.0, "l_extendedprice": 25340.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-09", "l_commitdate": "1994-07-02", "l_receiptdate": "1994-05-15", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "ckages sublate carefully closely regular " }
+{ "l_orderkey": 3076, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 31.0, "l_extendedprice": 28055.0, "l_discount": 0.06, "l_tax": 0.06, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-08-10", "l_commitdate": "1993-09-17", "l_receiptdate": "1993-08-17", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": "regular depos" }
+{ "l_orderkey": 3328, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 23.0, "l_extendedprice": 20815.0, "l_discount": 0.01, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-01-12", "l_commitdate": "1993-02-07", "l_receiptdate": "1993-01-30", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "y. careful" }
+{ "l_orderkey": 3555, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 4, "l_quantity": 19.0, "l_extendedprice": 17195.0, "l_discount": 0.0, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-08", "l_commitdate": "1996-09-14", "l_receiptdate": "1996-10-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "leep special theodolit" }
+{ "l_orderkey": 3649, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 1, "l_quantity": 25.0, "l_extendedprice": 22625.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-27", "l_commitdate": "1994-08-23", "l_receiptdate": "1994-11-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "special re" }
+{ "l_orderkey": 3680, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.0, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-01-06", "l_commitdate": "1993-03-02", "l_receiptdate": "1993-01-08", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "iously ironic platelets in" }
+{ "l_orderkey": 3877, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 5, "l_quantity": 41.0, "l_extendedprice": 37105.0, "l_discount": 0.03, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-06-30", "l_commitdate": "1993-07-20", "l_receiptdate": "1993-07-01", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "FOB", "l_comment": "integrate against the expres" }
+{ "l_orderkey": 3970, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 7, "l_quantity": 46.0, "l_extendedprice": 41630.0, "l_discount": 0.08, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-05-02", "l_commitdate": "1992-05-12", "l_receiptdate": "1992-05-10", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ix slyly. quickly silen" }
+{ "l_orderkey": 4229, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 34.0, "l_extendedprice": 30770.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-26", "l_commitdate": "1998-04-13", "l_receiptdate": "1998-06-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "thely final accounts use even packa" }
+{ "l_orderkey": 5283, "l_partkey": 5, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18100.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-16", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "al deposits? blithely even pinto beans" }
+{ "l_orderkey": 5665, "l_partkey": 5, "l_suppkey": 8, "l_linenumber": 2, "l_quantity": 14.0, "l_extendedprice": 12670.0, "l_discount": 0.02, "l_tax": 0.0, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-09-16", "l_receiptdate": "1993-07-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "- special pinto beans sleep quickly blithel" }
+{ "l_orderkey": 5959, "l_partkey": 5, "l_suppkey": 6, "l_linenumber": 3, "l_quantity": 4.0, "l_extendedprice": 3620.0, "l_discount": 0.04, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-14", "l_commitdate": "1992-07-05", "l_receiptdate": "1992-07-01", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "gular requests ar" }
+{ "l_orderkey": 260, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 29.0, "l_extendedprice": 26274.0, "l_discount": 0.1, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-03-15", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-04-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "fluffily even asymptotes. express wa" }
+{ "l_orderkey": 290, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 35.0, "l_extendedprice": 31710.0, "l_discount": 0.01, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-04-01", "l_commitdate": "1994-02-05", "l_receiptdate": "1994-04-27", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ove the final foxes detect slyly fluffily" }
+{ "l_orderkey": 768, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 30.0, "l_extendedprice": 27180.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-09-22", "l_commitdate": "1996-11-03", "l_receiptdate": "1996-10-13", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": " furiously fluffy pinto beans haggle along" }
+{ "l_orderkey": 801, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 1, "l_quantity": 13.0, "l_extendedprice": 11778.0, "l_discount": 0.1, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-25", "l_commitdate": "1992-04-24", "l_receiptdate": "1992-05-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "s are fluffily stealthily expres" }
+{ "l_orderkey": 1124, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 13.0, "l_extendedprice": 11778.0, "l_discount": 0.05, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-09-05", "l_commitdate": "1998-10-03", "l_receiptdate": "1998-09-30", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "t the slyly " }
+{ "l_orderkey": 1220, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32616.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-12", "l_commitdate": "1996-10-03", "l_receiptdate": "1996-12-15", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "unusual, silent pinto beans aga" }
+{ "l_orderkey": 1284, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-02-29", "l_commitdate": "1996-02-11", "l_receiptdate": "1996-03-01", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": " regular asymptotes. " }
+{ "l_orderkey": 1542, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 3, "l_quantity": 18.0, "l_extendedprice": 16308.0, "l_discount": 0.05, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-10-17", "l_commitdate": "1993-11-15", "l_receiptdate": "1993-10-26", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "pending instr" }
+{ "l_orderkey": 1638, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41676.0, "l_discount": 0.03, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-10-16", "l_commitdate": "1997-10-28", "l_receiptdate": "1997-11-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "otes haggle before the slyly bold instructi" }
+{ "l_orderkey": 1827, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 7, "l_quantity": 38.0, "l_extendedprice": 34428.0, "l_discount": 0.05, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-17", "l_commitdate": "1996-08-29", "l_receiptdate": "1996-11-07", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " blithely. express, bo" }
+{ "l_orderkey": 2049, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 39.0, "l_extendedprice": 35334.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-01-17", "l_commitdate": "1996-01-21", "l_receiptdate": "1996-02-03", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "the even pinto beans " }
+{ "l_orderkey": 2054, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 5, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.08, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-06-23", "l_commitdate": "1992-08-09", "l_receiptdate": "1992-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "n pinto beans. ironic courts are iro" }
+{ "l_orderkey": 2179, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-10-26", "l_commitdate": "1996-11-05", "l_receiptdate": "1996-11-16", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " cajole carefully. " }
+{ "l_orderkey": 2276, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 6, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-05", "l_commitdate": "1996-06-30", "l_receiptdate": "1996-08-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "s. deposits " }
+{ "l_orderkey": 2370, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 21.0, "l_extendedprice": 19026.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-01", "l_commitdate": "1994-02-19", "l_receiptdate": "1994-02-09", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "ecial dependencies must have to " }
+{ "l_orderkey": 2689, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 45.0, "l_extendedprice": 40770.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-29", "l_commitdate": "1992-06-22", "l_receiptdate": "1992-04-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "e quickly. carefully silent" }
+{ "l_orderkey": 3043, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 15.0, "l_extendedprice": 13590.0, "l_discount": 0.03, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1992-05-27", "l_commitdate": "1992-06-03", "l_receiptdate": "1992-06-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "FOB", "l_comment": "usly furiously" }
+{ "l_orderkey": 3137, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.06, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-10-01", "l_commitdate": "1995-09-11", "l_receiptdate": "1995-10-30", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "posits wake. silent excuses boost about" }
+{ "l_orderkey": 3329, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 9.0, "l_extendedprice": 8154.0, "l_discount": 0.0, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-24", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-08-01", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "lly final depo" }
+{ "l_orderkey": 3426, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 4, "l_quantity": 9.0, "l_extendedprice": 8154.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-24", "l_commitdate": "1997-01-14", "l_receiptdate": "1997-01-13", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": "pecial theodolites haggle fluf" }
+{ "l_orderkey": 4005, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 14.0, "l_extendedprice": 12684.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-27", "l_commitdate": "1997-01-09", "l_receiptdate": "1996-12-25", "l_shipinstruct": "NONE", "l_shipmode": "TRUCK", "l_comment": "ld requests. slyly final instructi" }
+{ "l_orderkey": 4036, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 46.0, "l_extendedprice": 41676.0, "l_discount": 0.09, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-06-21", "l_commitdate": "1997-05-29", "l_receiptdate": "1997-07-18", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "usly across the even th" }
+{ "l_orderkey": 4195, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 14.0, "l_extendedprice": 12684.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-06", "l_commitdate": "1993-07-21", "l_receiptdate": "1993-09-18", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "REG AIR", "l_comment": "ironic packages. carefully express" }
+{ "l_orderkey": 4387, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 6, "l_quantity": 40.0, "l_extendedprice": 36240.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-29", "l_commitdate": "1995-12-10", "l_receiptdate": "1995-12-20", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "deas according to the blithely regular fox" }
+{ "l_orderkey": 4483, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 1, "l_quantity": 32.0, "l_extendedprice": 28992.0, "l_discount": 0.07, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-05", "l_commitdate": "1992-05-25", "l_receiptdate": "1992-04-08", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "ests haggle. slyl" }
+{ "l_orderkey": 4485, "l_partkey": 6, "l_suppkey": 7, "l_linenumber": 5, "l_quantity": 47.0, "l_extendedprice": 42582.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1995-03-11", "l_commitdate": "1995-01-11", "l_receiptdate": "1995-03-21", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "luffily pending acc" }
+{ "l_orderkey": 4612, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 20.0, "l_extendedprice": 18120.0, "l_discount": 0.02, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-09-24", "l_commitdate": "1993-12-18", "l_receiptdate": "1993-10-22", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "beans sleep blithely iro" }
+{ "l_orderkey": 4676, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 33.0, "l_extendedprice": 29898.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-12-29", "l_commitdate": "1995-10-01", "l_receiptdate": "1996-01-18", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "yly express " }
+{ "l_orderkey": 4870, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 4, "l_quantity": 4.0, "l_extendedprice": 3624.0, "l_discount": 0.03, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-23", "l_commitdate": "1994-09-16", "l_receiptdate": "1994-11-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "its wake quickly. slyly quick" }
+{ "l_orderkey": 5124, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 2, "l_quantity": 41.0, "l_extendedprice": 37146.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1997-07-05", "l_commitdate": "1997-06-29", "l_receiptdate": "1997-07-23", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "RAIL", "l_comment": "wake across the" }
+{ "l_orderkey": 5125, "l_partkey": 6, "l_suppkey": 9, "l_linenumber": 1, "l_quantity": 38.0, "l_extendedprice": 34428.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-20", "l_commitdate": "1998-04-14", "l_receiptdate": "1998-03-22", "l_shipinstruct": "COLLECT COD", "l_shipmode": "MAIL", "l_comment": "ily even deposits w" }
+{ "l_orderkey": 5223, "l_partkey": 6, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 19.0, "l_extendedprice": 17214.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-10-28", "l_commitdate": "1994-08-26", "l_receiptdate": "1994-10-31", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ntly. furiously even excuses a" }
+{ "l_orderkey": 5760, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 24.0, "l_extendedprice": 21744.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-07-15", "l_commitdate": "1994-07-04", "l_receiptdate": "1994-08-08", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "s. bravely ironic accounts among" }
+{ "l_orderkey": 5957, "l_partkey": 6, "l_suppkey": 1, "l_linenumber": 6, "l_quantity": 41.0, "l_extendedprice": 37146.0, "l_discount": 0.1, "l_tax": 0.07, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-03-25", "l_commitdate": "1994-02-20", "l_receiptdate": "1994-03-31", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "es across the regular requests maint" }
+{ "l_orderkey": 579, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 4, "l_quantity": 41.0, "l_extendedprice": 37187.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-05-28", "l_commitdate": "1998-05-01", "l_receiptdate": "1998-06-04", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "bold, express requests sublate slyly. blith" }
+{ "l_orderkey": 771, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-18", "l_commitdate": "1995-08-02", "l_receiptdate": "1995-08-07", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "carefully. pending in" }
+{ "l_orderkey": 771, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 14.0, "l_extendedprice": 12698.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-07-31", "l_commitdate": "1995-08-13", "l_receiptdate": "1995-08-07", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "r, final packages are slyly iro" }
+{ "l_orderkey": 834, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 11.0, "l_extendedprice": 9977.0, "l_discount": 0.03, "l_tax": 0.0, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-09-18", "l_commitdate": "1994-08-03", "l_receiptdate": "1994-10-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "inst the regular packa" }
+{ "l_orderkey": 1281, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 37.0, "l_extendedprice": 33559.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-03-19", "l_commitdate": "1995-02-02", "l_receiptdate": "1995-03-27", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "ounts detect" }
+{ "l_orderkey": 2080, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 5.0, "l_extendedprice": 4535.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-08-26", "l_commitdate": "1993-08-07", "l_receiptdate": "1993-09-02", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "refully unusual theo" }
+{ "l_orderkey": 2150, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 6, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.09, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-08-27", "l_commitdate": "1994-08-22", "l_receiptdate": "1994-09-18", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "press platelets haggle until the slyly fi" }
+{ "l_orderkey": 2182, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 4, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.04, "l_tax": 0.07, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-05-08", "l_commitdate": "1994-06-02", "l_receiptdate": "1994-05-09", "l_shipinstruct": "COLLECT COD", "l_shipmode": "REG AIR", "l_comment": "ments are fu" }
+{ "l_orderkey": 2208, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 7, "l_quantity": 45.0, "l_extendedprice": 40815.0, "l_discount": 0.0, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-05-05", "l_commitdate": "1995-06-10", "l_receiptdate": "1995-05-11", "l_shipinstruct": "NONE", "l_shipmode": "SHIP", "l_comment": "e fluffily regular theodolites caj" }
+{ "l_orderkey": 2598, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 12.0, "l_extendedprice": 10884.0, "l_discount": 0.0, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-06-17", "l_commitdate": "1996-04-12", "l_receiptdate": "1996-06-24", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "express packages nag sly" }
+{ "l_orderkey": 2658, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 5, "l_quantity": 45.0, "l_extendedprice": 40815.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-11-02", "l_commitdate": "1995-11-08", "l_receiptdate": "1995-11-29", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "MAIL", "l_comment": "e special requests. quickly ex" }
+{ "l_orderkey": 2659, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 5, "l_quantity": 9.0, "l_extendedprice": 8163.0, "l_discount": 0.08, "l_tax": 0.03, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-02-07", "l_commitdate": "1994-03-17", "l_receiptdate": "1994-03-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "ly final packages sleep ac" }
+{ "l_orderkey": 2752, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 2, "l_quantity": 29.0, "l_extendedprice": 26303.0, "l_discount": 0.02, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-08", "l_receiptdate": "1994-01-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "gly blithely re" }
+{ "l_orderkey": 3140, "l_partkey": 7, "l_suppkey": 4, "l_linenumber": 1, "l_quantity": 21.0, "l_extendedprice": 19047.0, "l_discount": 0.08, "l_tax": 0.02, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-04-12", "l_commitdate": "1992-05-31", "l_receiptdate": "1992-04-21", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": " furiously sly excuses according to the" }
+{ "l_orderkey": 3204, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 2, "l_quantity": 39.0, "l_extendedprice": 35373.0, "l_discount": 0.1, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-11", "l_commitdate": "1993-03-19", "l_receiptdate": "1993-02-28", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": "sits sleep theodolites. slyly bo" }
+{ "l_orderkey": 4166, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 17.0, "l_extendedprice": 15419.0, "l_discount": 0.02, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-29", "l_commitdate": "1993-05-15", "l_receiptdate": "1993-07-24", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ackages. re" }
+{ "l_orderkey": 4614, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 19.0, "l_extendedprice": 17233.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-05-17", "l_commitdate": "1996-06-21", "l_receiptdate": "1996-06-08", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "ix. carefully regular " }
+{ "l_orderkey": 5318, "l_partkey": 7, "l_suppkey": 10, "l_linenumber": 3, "l_quantity": 37.0, "l_extendedprice": 33559.0, "l_discount": 0.07, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-09", "l_commitdate": "1993-06-22", "l_receiptdate": "1993-07-21", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "ickly final deposi" }
+{ "l_orderkey": 5351, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32652.0, "l_discount": 0.06, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-27", "l_commitdate": "1998-07-06", "l_receiptdate": "1998-08-25", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "ss the ironic, regular asymptotes cajole " }
+{ "l_orderkey": 5606, "l_partkey": 7, "l_suppkey": 2, "l_linenumber": 5, "l_quantity": 25.0, "l_extendedprice": 22675.0, "l_discount": 0.06, "l_tax": 0.0, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-12-25", "l_commitdate": "1997-01-12", "l_receiptdate": "1997-01-11", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "breach about the furiously bold " }
+{ "l_orderkey": 5670, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 24.0, "l_extendedprice": 21768.0, "l_discount": 0.09, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1993-07-17", "l_commitdate": "1993-07-01", "l_receiptdate": "1993-08-03", "l_shipinstruct": "NONE", "l_shipmode": "AIR", "l_comment": "press, express requests haggle" }
+{ "l_orderkey": 5794, "l_partkey": 7, "l_suppkey": 8, "l_linenumber": 3, "l_quantity": 15.0, "l_extendedprice": 13605.0, "l_discount": 0.09, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-06-25", "l_commitdate": "1993-06-27", "l_receiptdate": "1993-07-09", "l_shipinstruct": "NONE", "l_shipmode": "MAIL", "l_comment": "blithely regular ideas. final foxes haggle " }
+{ "l_orderkey": 68, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 1, "l_quantity": 3.0, "l_extendedprice": 2724.0, "l_discount": 0.05, "l_tax": 0.02, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-07-04", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-07-21", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "fully special instructions cajole. furious" }
+{ "l_orderkey": 225, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 5, "l_quantity": 31.0, "l_extendedprice": 28148.0, "l_discount": 0.04, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1995-06-21", "l_commitdate": "1995-07-24", "l_receiptdate": "1995-07-04", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "FOB", "l_comment": "special platelets. quickly r" }
+{ "l_orderkey": 230, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 3, "l_quantity": 1.0, "l_extendedprice": 908.0, "l_discount": 0.07, "l_tax": 0.06, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-01-22", "l_commitdate": "1994-01-03", "l_receiptdate": "1994-02-05", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "RAIL", "l_comment": "blithely unusual dolphins. bold, ex" }
+{ "l_orderkey": 1060, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 26.0, "l_extendedprice": 23608.0, "l_discount": 0.06, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-04-12", "l_commitdate": "1993-04-01", "l_receiptdate": "1993-04-20", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "counts; even deposits are carefull" }
+{ "l_orderkey": 1222, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 26.0, "l_extendedprice": 23608.0, "l_discount": 0.02, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1993-02-13", "l_commitdate": "1993-03-20", "l_receiptdate": "1993-02-22", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "MAIL", "l_comment": ", even accounts are ironic" }
+{ "l_orderkey": 1283, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 6, "l_quantity": 30.0, "l_extendedprice": 27240.0, "l_discount": 0.06, "l_tax": 0.07, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-22", "l_commitdate": "1996-11-22", "l_receiptdate": "1996-12-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "TRUCK", "l_comment": "t the fluffily" }
+{ "l_orderkey": 1472, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-11-06", "l_commitdate": "1996-11-13", "l_receiptdate": "1996-11-12", "l_shipinstruct": "COLLECT COD", "l_shipmode": "SHIP", "l_comment": "riously silent deposits to the pending d" }
+{ "l_orderkey": 1540, "l_partkey": 8, "l_suppkey": 3, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22700.0, "l_discount": 0.08, "l_tax": 0.04, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1992-11-15", "l_commitdate": "1992-10-24", "l_receiptdate": "1992-12-14", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "SHIP", "l_comment": "ironic deposits amo" }
+{ "l_orderkey": 1760, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 2, "l_quantity": 3.0, "l_extendedprice": 2724.0, "l_discount": 0.0, "l_tax": 0.06, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-18", "l_commitdate": "1996-07-01", "l_receiptdate": "1996-08-01", "l_shipinstruct": "NONE", "l_shipmode": "RAIL", "l_comment": "lyly bold dolphins haggle carefully. sl" }
+{ "l_orderkey": 1762, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 5, "l_quantity": 49.0, "l_extendedprice": 44492.0, "l_discount": 0.08, "l_tax": 0.05, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-10-20", "l_commitdate": "1994-11-02", "l_receiptdate": "1994-11-10", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "SHIP", "l_comment": " packages sleep fluffily pen" }
+{ "l_orderkey": 1828, "l_partkey": 8, "l_suppkey": 3, "l_linenumber": 4, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.02, "l_tax": 0.05, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-15", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-05-28", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": " accounts run slyly " }
+{ "l_orderkey": 2048, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 5.0, "l_extendedprice": 4540.0, "l_discount": 0.04, "l_tax": 0.04, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1994-01-18", "l_commitdate": "1994-02-01", "l_receiptdate": "1994-01-29", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "TRUCK", "l_comment": "affix carefully against " }
+{ "l_orderkey": 2277, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 2.0, "l_extendedprice": 1816.0, "l_discount": 0.1, "l_tax": 0.08, "l_returnflag": "A", "l_linestatus": "F", "l_shipdate": "1995-02-01", "l_commitdate": "1995-02-04", "l_receiptdate": "1995-03-02", "l_shipinstruct": "TAKE BACK RETURN", "l_shipmode": "AIR", "l_comment": "endencies sleep idly pending p" }
+{ "l_orderkey": 3777, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 2, "l_quantity": 10.0, "l_extendedprice": 9080.0, "l_discount": 0.03, "l_tax": 0.01, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-05-22", "l_commitdate": "1994-05-29", "l_receiptdate": "1994-06-13", "l_shipinstruct": "COLLECT COD", "l_shipmode": "RAIL", "l_comment": "le. ironic depths a" }
+{ "l_orderkey": 4291, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 25.0, "l_extendedprice": 22700.0, "l_discount": 0.09, "l_tax": 0.08, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-02-14", "l_commitdate": "1994-02-08", "l_receiptdate": "1994-03-15", "l_shipinstruct": "COLLECT COD", "l_shipmode": "AIR", "l_comment": "uctions. furiously regular ins" }
+{ "l_orderkey": 4322, "l_partkey": 8, "l_suppkey": 9, "l_linenumber": 3, "l_quantity": 12.0, "l_extendedprice": 10896.0, "l_discount": 0.09, "l_tax": 0.05, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-03-29", "l_commitdate": "1998-06-05", "l_receiptdate": "1998-04-16", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "TRUCK", "l_comment": "e blithely against the slyly unusu" }
+{ "l_orderkey": 4420, "l_partkey": 8, "l_suppkey": 5, "l_linenumber": 1, "l_quantity": 7.0, "l_extendedprice": 6356.0, "l_discount": 0.07, "l_tax": 0.03, "l_returnflag": "R", "l_linestatus": "F", "l_shipdate": "1994-08-30", "l_commitdate": "1994-09-03", "l_receiptdate": "1994-09-25", "l_shipinstruct": "NONE", "l_shipmode": "FOB", "l_comment": " regular instructions sleep around" }
+{ "l_orderkey": 4613, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 4, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.04, "l_tax": 0.01, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1998-04-22", "l_commitdate": "1998-05-05", "l_receiptdate": "1998-05-04", "l_shipinstruct": "DELIVER IN PERSON", "l_shipmode": "AIR", "l_comment": "gainst the furiously ironic" }
+{ "l_orderkey": 4614, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 3, "l_quantity": 36.0, "l_extendedprice": 32688.0, "l_discount": 0.1, "l_tax": 0.04, "l_returnflag": "N", "l_linestatus": "O", "l_shipdate": "1996-07-05", "l_commitdate": "1996-06-26", "l_receiptdate": "1996-07-07", "l_shipinstruct": "NONE", "l_shipmode": "REG AIR", "l_comment": "onic foxes affix furi" }
+{ "l_orderkey": 5057, "l_partkey": 8, "l_suppkey": 1, "l_linenumber": 2, "l_quantity": 45.0, "l_extendedprice": 40860.0, "l_discount": 0.08, "l_tax": 0.07, "l_

<TRUNCATED>

[17/21] asterixdb git commit: [ASTERIXDB-2286][COMP][FUN][HYR] Parallel Sort Optimization

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5_ps.sqlpp
new file mode 100644
index 0000000..6ef533a
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_5_ps.sqlpp
@@ -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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6_ps.sqlpp
new file mode 100644
index 0000000..4c4d754
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_as_or_6_ps.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.
+ */
+
+// requesttype=application/json
+
+// param $p1:string=Customer#000000001
+// param $p2:string=Customer#000000002
+// param $p3:string=Customer#000000003
+
+drop  dataverse tpch if exists;
+create  dataverse tpch;
+use tpch;
+
+create type tpch.CustomerType as closed {
+  c_custkey : bigint,
+  c_name : string,
+  c_address : string,
+  c_nationkey : bigint,
+  c_phone : string,
+  c_acctbal : double,
+  c_mktsegment : string,
+  c_comment : string
+};
+
+create  dataset Customer(CustomerType) primary key c_custkey;
+
+set rewrite_in_as_or "false";
+set `compiler.sort.parallel` "true";
+
+select value c.c_custkey
+from Customer c
+where c.c_name in [$p1, $p2, $p3]
+order by c.c_custkey;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_correlated_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_correlated_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_correlated_ps.sqlpp
new file mode 100644
index 0000000..8d852f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_correlated_ps.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE OrderType AS {
+  o_orderkey: integer
+};
+
+CREATE TYPE CustomerType AS {
+  c_custkey: integer
+};
+
+CREATE COLLECTION Customers(CustomerType) PRIMARY KEY c_custkey;
+CREATE COLLECTION Orders(OrderType) PRIMARY KEY o_orderkey;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT c.c_custkey customer_name
+FROM Customers c
+WHERE c.c_custkey IN (
+    SELECT VALUE o.o_custkey
+    FROM Orders o
+    WHERE c.c_custkey = o.o_custkey
+  )  AND c.c_nationkey = 5
+ORDER BY customer_name;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_ps.sqlpp
new file mode 100644
index 0000000..876ffbd
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/in_ps.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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE OrderType AS {
+  o_orderkey: integer
+};
+
+CREATE TYPE CustomerType AS {
+  c_custkey: integer
+};
+
+
+CREATE COLLECTION Customers(CustomerType) PRIMARY KEY c_custkey;
+CREATE COLLECTION Orders(OrderType) PRIMARY KEY o_orderkey;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT c.c_custkey customer_name
+FROM Customers c
+WHERE c.c_custkey IN (
+    SELECT VALUE o.o_custkey
+    FROM Orders o
+  )  AND c.c_nationkey = 5
+ORDER BY customer_name;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists_ps.sqlpp
new file mode 100644
index 0000000..3fcaee3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/subquery/not_exists_ps.sqlpp
@@ -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.
+ */
+
+DROP DATAVERSE test IF EXISTS;
+CREATE DATAVERSE test;
+USE test;
+
+CREATE TYPE OrderType AS CLOSED {
+  o_orderkey: integer,
+  o_custkey: integer,
+  o_orderstatus: string,
+  o_totalprice: double,
+  o_orderdate: string,
+  o_orderpriority: string,
+  o_clerk: string,
+  o_shippriority: integer,
+  o_comment: string
+};
+
+CREATE TYPE CustomerType AS CLOSED {
+  c_custkey: integer,
+  c_name: string,
+  c_address: string,
+  c_nationkey: integer,
+  c_phone: string,
+  c_acctbal: double,
+  c_mktsegment: string,
+  c_comment: string
+};
+
+CREATE EXTERNAL DATASET Customer(CustomerType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/customer.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+CREATE EXTERNAL DATASET Orders(OrderType) USING `localfs`
+((`path`=`asterix_nc1://data/tpch0.001/orders.tbl`),
+(`input-format`=`text-input-format`),(`format`=`delimited-text`),(`delimiter`=`|`));
+
+SET `compiler.sort.parallel` "true";
+
+WITH q22_customer_tmp AS
+(
+    SELECT c_acctbal, c_custkey, substring(c_phone,0,2) AS cntrycode
+    FROM  Customer
+)
+,
+avg AS (
+        SELECT ELEMENT AVG(c_acctbal)
+        FROM  Customer
+        WHERE c_acctbal > 0.0
+)[0]
+SELECT  cntrycode, count(ct) AS numcust, SUM(c_acctbal) AS totacctbal
+FROM  q22_customer_tmp AS ct
+WHERE c_acctbal > avg
+      AND NOT EXISTS (SELECT * FROM Orders o WHERE o.o_custkey = ct.c_custkey)
+GROUP BY cntrycode
+ORDER BY cntrycode;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp
new file mode 100644
index 0000000..efe7b00
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpcds/query-ASTERIXDB-1596_ps.sqlpp
@@ -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.
+ */
+
+drop dataverse tpcds if exists;
+create dataverse tpcds;
+use tpcds;
+
+create type tpcds.catalog_sales_type as closed {
+    cs_sold_date_sk:           bigint?,
+    cs_sold_time_sk:           bigint?,
+    cs_ship_date_sk:           bigint?,
+    cs_bill_customer_sk:       bigint?,
+    cs_bill_cdemo_sk:          bigint?,
+    cs_bill_hdemo_sk:          bigint?,
+    cs_bill_addr_sk:           bigint?,
+    cs_ship_customer_sk:       bigint?,
+    cs_ship_cdemo_sk:          bigint?,
+    cs_ship_hdemo_sk:          bigint?,
+    cs_ship_addr_sk:           bigint?,
+    cs_call_center_sk:         bigint?,
+    cs_catalog_page_sk:        bigint?,
+    cs_ship_mode_sk:           bigint?,
+    cs_warehouse_sk:           bigint?,
+    cs_item_sk:                bigint,
+    cs_promo_sk:               bigint?,
+    cs_order_number:           bigint,
+    cs_quantity:               bigint?,
+    cs_wholesale_cost:         double?,
+    cs_list_price:             double?,
+    cs_sales_price:            double?,
+    cs_ext_discount_amt:       double?,
+    cs_ext_sales_price:        double?,
+    cs_ext_wholesale_cost:     double?,
+    cs_ext_list_price:         double?,
+    cs_ext_tax:                double?,
+    cs_coupon_amt:             double?,
+    cs_ext_ship_cost:          double?,
+    cs_net_paid:               double?,
+    cs_net_paid_inc_tax:       double?,
+    cs_net_paid_inc_ship:      double?,
+    cs_net_paid_inc_ship_tax:  double?,
+    cs_net_profit:             double?
+};
+
+create type tpcds.catalog_returns_type as closed {
+    cr_returned_date_sk:       bigint?,
+    cr_returned_time_sk:       bigint?,
+    cr_item_sk:                bigint,
+    cr_refunded_customer_sk:   bigint?,
+    cr_refunded_cdemo_sk:      bigint?,
+    cr_refunded_hdemo_sk:      bigint?,
+    cr_refunded_addr_sk:       bigint?,
+    cr_returning_customer_sk:  bigint?,
+    cr_returning_cdemo_sk:     bigint?,
+    cr_returning_hdemo_sk:     bigint?,
+    cr_returning_addr_sk:      bigint?,
+    cr_call_center_sk:         bigint?,
+    cr_catalog_page_sk:        bigint?,
+    cr_ship_mode_sk:           bigint?,
+    cr_warehouse_sk:           bigint?,
+    cr_reason_sk:              bigint?,
+    cr_order_number:           bigint,
+    cr_return_quantity:        bigint?,
+    cr_return_amount:          double?,
+    cr_return_tax:             double?,
+    cr_return_amt_inc_tax:     double?,
+    cr_fee:                    double?,
+    cr_return_ship_cost:       double?,
+    cr_refunded_cash:          double?,
+    cr_reversed_charge:        double?,
+    cr_store_credit:           double?,
+    cr_net_loss:               double?
+};
+
+create type tpcds.item_type as closed {
+    i_item_sk:                 bigint,
+    i_item_id:                 string,
+    i_rec_start_date:          string?,
+    i_rec_end_date:            string?,
+    i_item_desc:               string?,
+    i_current_price:           double?,
+    i_wholesale_cost:          double?,
+    i_brand_id:                bigint? ,
+    i_brand:                   string?,
+    i_class_id:                bigint? ,
+    i_class:                   string?,
+    i_category_id:             bigint? ,
+    i_category:                string?,
+    i_manufact_id:             bigint? ,
+    i_manufact:                string?,
+    i_size:                    string?,
+    i_formulation:             string?,
+    i_color:                   string?,
+    i_units:                   string?,
+    i_container:               string?,
+    i_manager_id:              bigint?,
+    i_product_name:            string?
+};
+
+create dataset catalog_sales (catalog_sales_type) primary key cs_item_sk, cs_order_number;
+create dataset catalog_returns (catalog_returns_type) primary key cr_item_sk, cr_order_number;
+create dataset item (item_type) primary key i_item_sk;
+
+SET `compiler.sort.parallel` "true";
+
+SELECT *
+FROM catalog_sales cs1
+     LEFT OUTER JOIN catalog_returns cr1
+     ON (cs1.cs_order_number = cr1.cr_order_number
+         AND cs1.cs_item_sk = cr1.cr_item_sk),
+     item i1
+WHERE i1.i_item_sk = cs1.cs_item_sk
+ORDER BY cs1.cs_item_sk, cs1.cs_order_number;
+
+drop dataverse tpcds;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_broadcast_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_broadcast_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_broadcast_ps.sqlpp
new file mode 100644
index 0000000..6c55128
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_broadcast_ps.sqlpp
@@ -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.
+ */
+
+DROP dataverse tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS {
+  l_linenumber : integer
+};
+
+CREATE TYPE OrderType AS {
+  o_orderkey : integer
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_linenumber;
+CREATE DATASET Orders(OrderType) PRIMARY KEY o_orderkey;
+
+SET `compiler.sort.parallel` "true";
+
+/** The plan tests that the expression for different switch-case branches are not extracted.*/
+SELECT l.l_shipmode,
+       sum(CASE
+             WHEN o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH' THEN 1 +  o.o_orderpriority * 0
+             ELSE 0 + o.o_orderpriority * 0
+           END) high_line_count,
+       sum(CASE o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH'
+             WHEN true THEN 0 + o.o_orderpriority * 0
+             ELSE 1 +  o.o_orderpriority * 0
+           END) low_line_count
+FROM  LineItem l,
+      Orders o
+WHERE l.l_orderkey /*+ bcast */ = o.o_orderkey AND l.l_commitdate < l.l_receiptdate AND
+      l.l_shipdate < l.l_commitdate AND l.l_receiptdate >= '1994-01-01' AND
+      l.l_receiptdate < '1995-01-01' AND (l.l_shipmode = 'MAIL' OR l.l_shipmode = 'SHIP')
+GROUP BY l.l_shipmode
+ORDER BY l.l_shipmode;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_ps.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_ps.sqlpp b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_ps.sqlpp
new file mode 100644
index 0000000..62a9c23
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/queries/tpch/q12_shipping_ps.sqlpp
@@ -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.
+ */
+
+DROP dataverse tpch IF EXISTS;
+CREATE  dataverse tpch;
+USE tpch;
+
+CREATE TYPE LineItemType AS {
+  l_linenumber : integer
+};
+
+CREATE TYPE OrderType AS {
+  o_orderkey : integer
+};
+
+CREATE DATASET LineItem(LineItemType) PRIMARY KEY l_linenumber;
+CREATE DATASET Orders(OrderType) PRIMARY KEY o_orderkey;
+
+SET `compiler.sort.parallel` "true";
+
+/** The plan tests that the expression for different switch-case branches are not extracted.*/
+SELECT l.l_shipmode,
+       sum(CASE
+             WHEN o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH' THEN 1 +  o.o_orderpriority * 0
+             ELSE 0 + o.o_orderpriority * 0
+           END) high_line_count,
+       sum(CASE o.o_orderpriority = '1-URGENT' or o.o_orderpriority = '2-HIGH'
+             WHEN true THEN 0 + o.o_orderpriority * 0
+             ELSE 1 +  o.o_orderpriority * 0
+           END) low_line_count
+FROM  LineItem l,
+      Orders o
+WHERE o.o_orderkey = l.l_orderkey AND l.l_commitdate < l.l_receiptdate AND
+      l.l_shipdate < l.l_commitdate AND l.l_receiptdate >= '1994-01-01' AND
+      l.l_receiptdate < '1995-01-01' AND (l.l_shipmode = 'MAIL' OR l.l_shipmode = 'SHIP')
+GROUP BY l.l_shipmode
+ORDER BY l.l_shipmode;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..4966390
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_01_ps.plan
@@ -0,0 +1,109 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$47(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$47(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), $$39(ASC)]  |PARTITIONED|
+                            -- HASH_PARTITION_EXCHANGE [$$64]  |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  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- SPLIT  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                          -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                              -- ASSIGN  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH  |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|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |PARTITIONED|
+                                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |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[$$64]  |PARTITIONED|
+                                        {
+                                          -- AGGREGATE  |LOCAL|
+                                            -- STREAM_SELECT  |LOCAL|
+                                              -- NESTED_TUPLE_SOURCE  |LOCAL|
+                                        }
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$64(ASC), $$39(ASC)]  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$64]  |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  |PARTITIONED|
+                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                          -- STREAM_PROJECT  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- SPLIT  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- BTREE_SEARCH  |PARTITIONED|
+                                                                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                      -- STREAM_PROJECT  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- BTREE_SEARCH  |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|
+                                                -- SPLIT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |PARTITIONED|
+                                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..1ad81d2
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/leftouterjoin-probe-pidx-with-join-btree-sidx_02_ps.plan
@@ -0,0 +1,77 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$52(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$52(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|
+                                -- STREAM_PROJECT  |PARTITIONED|
+                                  -- ASSIGN  |PARTITIONED|
+                                    -- STREAM_PROJECT  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                      -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                        -- STREAM_PROJECT  |PARTITIONED|
+                                                          -- ASSIGN  |PARTITIONED|
+                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                              -- BTREE_SEARCH  |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|
+                                          -- STREAM_PROJECT  |PARTITIONED|
+                                            -- ASSIGN  |PARTITIONED|
+                                              -- STREAM_PROJECT  |PARTITIONED|
+                                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                  -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                                                  -- STREAM_PROJECT  |PARTITIONED|
+                                                                    -- ASSIGN  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- BTREE_SEARCH  |PARTITIONED|
+                                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                            -- ASSIGN  |PARTITIONED|
+                                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
index 71ee3db..9594dc1 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/secondary-self-equi-join-index-only.plan
@@ -23,12 +23,10 @@
                                             -- STREAM_PROJECT  |PARTITIONED|
                                               -- ASSIGN  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
-                                                  -- ASSIGN  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
                                                       -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                        -- DATASOURCE_SCAN  |PARTITIONED|
-                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
                 -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                   -- STREAM_PROJECT  |PARTITIONED|
                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
@@ -39,9 +37,7 @@
                               -- STREAM_PROJECT  |PARTITIONED|
                                 -- ASSIGN  |PARTITIONED|
                                   -- STREAM_PROJECT  |PARTITIONED|
-                                    -- ASSIGN  |PARTITIONED|
-                                      -- STREAM_PROJECT  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- DATASOURCE_SCAN  |PARTITIONED|
                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                          -- DATASOURCE_SCAN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan
index 0ad315e..2f03f0f 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index-join/sidx-non-idxonly-to-sidx-idxonly-equi-join_01.plan
@@ -24,19 +24,18 @@
                                               -- BROADCAST_EXCHANGE  |PARTITIONED|
                                                 -- STREAM_PROJECT  |PARTITIONED|
                                                   -- STREAM_SELECT  |PARTITIONED|
-                                                    -- STREAM_PROJECT  |PARTITIONED|
-                                                      -- ASSIGN  |PARTITIONED|
-                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                          -- BTREE_SEARCH  |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  |PARTITIONED|
-                                                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                                          -- ASSIGN  |PARTITIONED|
-                                                                            -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                                    -- ASSIGN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- ASSIGN  |PARTITIONED|
+                                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
               -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                 -- STREAM_PROJECT  |PARTITIONED|
                   -- ASSIGN  |PARTITIONED|
@@ -50,16 +49,15 @@
                                   -- BROADCAST_EXCHANGE  |PARTITIONED|
                                     -- STREAM_PROJECT  |PARTITIONED|
                                       -- STREAM_SELECT  |PARTITIONED|
-                                        -- STREAM_PROJECT  |PARTITIONED|
-                                          -- ASSIGN  |PARTITIONED|
-                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                              -- BTREE_SEARCH  |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  |PARTITIONED|
-                                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                                                              -- ASSIGN  |PARTITIONED|
-                                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |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  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7043901
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-secondary-68_ps.plan
@@ -0,0 +1,69 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$25(ASC), $$26(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$25(ASC), $$26(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  |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  |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  |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|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- BTREE_SEARCH  |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  |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  |PARTITIONED|
+                                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                            -- ASSIGN  |PARTITIONED|
+                                                              -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..53589ea
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01-disable-idxonly_ps.plan
@@ -0,0 +1,47 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(ASC)]  |PARTITIONED|
+              -- FORWARD  |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  |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  |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  |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  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- ASSIGN  |PARTITIONED|
+                                                          -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..7fe5795
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/btree-index/btree-sidx-idxonly-01_ps.plan
@@ -0,0 +1,65 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ASSIGN  |PARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+          -- STABLE_SORT [$$18(ASC)]  |PARTITIONED|
+            -- RANGE_PARTITION_EXCHANGE [$$18(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  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- SPLIT  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- BTREE_SEARCH  |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|
+                              -- SPLIT  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |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|
+                                      -- STREAM_PROJECT  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- SPLIT  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- BTREE_SEARCH  |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|
+                                        -- SPLIT  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- BTREE_SEARCH  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/fullparallelsort.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/fullparallelsort.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/fullparallelsort.plan
new file mode 100644
index 0000000..7763106
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/fullparallelsort.plan
@@ -0,0 +1,117 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$33(ASC)] SPLIT COUNT:  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_PROJECT  |PARTITIONED|
+                      -- STREAM_SELECT  |PARTITIONED|
+                        -- STREAM_PROJECT  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- PRE_CLUSTERED_GROUP_BY[$$29]  |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 [$$29][$$32]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- DATASOURCE_SCAN  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                      -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                        -- ASSIGN  |PARTITIONED|
+                                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                            -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                              -- RANGE_PARTITION_EXCHANGE [$$32(ASC)] SPLIT COUNT:  |PARTITIONED|
+                                                -- FORWARD  |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  |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|
+                                                                  -- ASSIGN  |PARTITIONED|
+                                                                    -- STREAM_PROJECT  |PARTITIONED|
+                                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                        -- DATASOURCE_SCAN  |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|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- PRE_CLUSTERED_GROUP_BY[$$29]  |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 [$$29][$$32]  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                                              -- HASH_PARTITION_EXCHANGE [$$32]  |PARTITIONED|
+                                                -- ASSIGN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                                      -- RANGE_PARTITION_EXCHANGE [$$32(ASC)] SPLIT COUNT:  |PARTITIONED|
+                                                        -- FORWARD  |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  |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|
+                                                                          -- ASSIGN  |PARTITIONED|
+                                                                            -- STREAM_PROJECT  |PARTITIONED|
+                                                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/nofullparallel_hint.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/nofullparallel_hint.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/nofullparallel_hint.plan
new file mode 100644
index 0000000..7262545
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/hints/nofullparallel_hint.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- SORT_MERGE_EXCHANGE [$$33(ASC) ]  |PARTITIONED|
+        -- STABLE_SORT [$$33(ASC)]  |PARTITIONED|
+          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+            -- STREAM_PROJECT  |PARTITIONED|
+              -- STREAM_SELECT  |PARTITIONED|
+                -- STREAM_PROJECT  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- PRE_CLUSTERED_GROUP_BY[$$29]  |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 [$$29][$$32]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- DATASOURCE_SCAN  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
+                              -- HASH_PARTITION_MERGE_EXCHANGE MERGE:[$$32(ASC)] HASH:[$$32]  |PARTITIONED|
+                                -- ASSIGN  |PARTITIONED|
+                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                    -- STABLE_SORT [$$32(ASC)]  |PARTITIONED|
+                                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                        -- STREAM_PROJECT  |PARTITIONED|
+                                          -- ASSIGN  |PARTITIONED|
+                                            -- STREAM_PROJECT  |PARTITIONED|
+                                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                -- DATASOURCE_SCAN  |PARTITIONED|
+                                                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                                    -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic_ps.plan
new file mode 100644
index 0000000..86c5aac
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains_ps.plan
new file mode 100644
index 0000000..0a9089e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/ngram-contains_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$13(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$13(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$17(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
new file mode 100644
index 0000000..173b7de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check_ps.plan
new file mode 100644
index 0000000..07563e4
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-check_ps.plan
@@ -0,0 +1,38 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- BTREE_SEARCH  |PARTITIONED|
+                          -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                            -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- BTREE_SEARCH  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- STABLE_SORT [$$19(ASC)]  |PARTITIONED|
+                                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                          -- LENGTH_PARTITIONED_INVERTED_INDEX_SEARCH  |PARTITIONED|
+                                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                              -- ASSIGN  |PARTITIONED|
+                                                -- EMPTY_TUPLE_SOURCE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/80225e2c/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic_ps.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic_ps.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic_ps.plan
new file mode 100644
index 0000000..173b7de
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/inverted-index-basic/olist-edit-distance-panic_ps.plan
@@ -0,0 +1,28 @@
+-- DISTRIBUTE_RESULT  |PARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+    -- STREAM_PROJECT  |PARTITIONED|
+      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+        -- STABLE_SORT [$$15(ASC)]  |PARTITIONED|
+          -- RANGE_PARTITION_EXCHANGE [$$15(ASC)]  |PARTITIONED|
+            -- FORWARD  |PARTITIONED|
+              -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                -- REPLICATE  |PARTITIONED|
+                  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                    -- STREAM_SELECT  |PARTITIONED|
+                      -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                        -- DATASOURCE_SCAN  |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|
+                                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                  -- DATASOURCE_SCAN  |PARTITIONED|
+                                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
+                                      -- EMPTY_TUPLE_SOURCE  |PARTITIONED|