You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@asterixdb.apache.org by bu...@apache.org on 2016/12/22 01:04:25 UTC

[1/8] asterixdb git commit: Supports flexible degree of parallelism.

Repository: asterixdb
Updated Branches:
  refs/heads/master 35ea53b0a -> 03e940a77


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
index ce8a704..f6a1bc4 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/AbstractHashJoinPOperator.java
@@ -30,16 +30,18 @@ import org.apache.hyracks.algebricks.core.algebra.base.EquivalenceClass;
 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.LogicalVariable;
-import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractLogicalOperator;
+import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBinaryJoinOperator.JoinKind;
 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.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.PhysicalRequirements;
+import org.apache.hyracks.algebricks.core.algebra.properties.RandomPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 import org.apache.hyracks.algebricks.core.algebra.properties.UnorderedPartitionedProperty;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 
 public abstract class AbstractHashJoinPOperator extends AbstractJoinPOperator {
 
@@ -85,36 +87,32 @@ public abstract class AbstractHashJoinPOperator extends AbstractJoinPOperator {
     }
 
     @Override
-    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator iop,
+    public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
             IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
-        StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
         // In a cost-based optimizer, we would also try to propagate the
         // parent's partitioning requirements.
-        AbstractLogicalOperator op = (AbstractLogicalOperator) iop;
-
-        IPartitioningProperty pp1 = null;
-        IPartitioningProperty pp2 = null;
-        if (op.getExecutionMode() == AbstractLogicalOperator.ExecutionMode.PARTITIONED) {
-            switch (partitioningType) {
-                case PAIRWISE: {
-                    pp1 = new UnorderedPartitionedProperty(new ListSet<LogicalVariable>(keysLeftBranch),
+        IPartitioningProperty pp1;
+        IPartitioningProperty pp2;
+        switch (partitioningType) {
+            case PAIRWISE:
+                pp1 = new UnorderedPartitionedProperty(new ListSet<>(keysLeftBranch),
                             context.getComputationNodeDomain());
-                    pp2 = new UnorderedPartitionedProperty(new ListSet<LogicalVariable>(keysRightBranch),
+                pp2 = new UnorderedPartitionedProperty(new ListSet<>(keysRightBranch),
                             context.getComputationNodeDomain());
                     break;
-                }
-                case BROADCAST: {
-                    pp2 = new BroadcastPartitioningProperty(context.getComputationNodeDomain());
-                    break;
-                }
-                default: {
-                    throw new IllegalStateException();
-                }
-            }
+            case BROADCAST:
+                pp1 = new RandomPartitioningProperty(context.getComputationNodeDomain());
+                pp2 = new BroadcastPartitioningProperty(context.getComputationNodeDomain());
+                break;
+            default:
+                throw new IllegalStateException();
         }
 
-        pv[0] = new StructuralPropertiesVector(pp1, null);
-        pv[1] = new StructuralPropertiesVector(pp2, null);
+        StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
+        pv[0] = OperatorPropertiesUtil.checkUnpartitionedAndGetPropertiesVector(op,
+                new StructuralPropertiesVector(pp1, null));
+        pv[1] = OperatorPropertiesUtil.checkUnpartitionedAndGetPropertiesVector(op,
+                new StructuralPropertiesVector(pp2, null));
 
         IPartitioningRequirementsCoordinator prc;
         switch (kind) {
@@ -180,14 +178,14 @@ public abstract class AbstractHashJoinPOperator extends AbstractJoinPOperator {
                                     }
                                     UnorderedPartitionedProperty upp2 =
                                             new UnorderedPartitionedProperty(modifuppreq, requirements.getNodeDomain());
-                                    return new Pair<Boolean, IPartitioningProperty>(false, upp2);
+                                    return new Pair<>(false, upp2);
                                 }
                                 case ORDERED_PARTITIONED: {
                                     throw new NotImplementedException();
                                 }
                             }
                         }
-                        return new Pair<Boolean, IPartitioningProperty>(true, requirements);
+                        return new Pair<>(true, requirements);
                     }
                 };
                 break;
@@ -196,7 +194,6 @@ public abstract class AbstractHashJoinPOperator extends AbstractJoinPOperator {
                 throw new IllegalStateException();
             }
         }
-
         return new PhysicalRequirements(pv, prc);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
index 574038b..80ed8fd 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/NestedLoopJoinPOperator.java
@@ -34,7 +34,9 @@ import org.apache.hyracks.algebricks.core.algebra.properties.IPartitioningProper
 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.RandomPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenContext;
 import org.apache.hyracks.algebricks.core.jobgen.impl.JobGenHelper;
 import org.apache.hyracks.algebricks.data.IBinaryBooleanInspector;
@@ -115,9 +117,10 @@ public class NestedLoopJoinPOperator extends AbstractJoinPOperator {
         StructuralPropertiesVector[] pv = new StructuralPropertiesVector[2];
 
         // TODO: leverage statistics to make better decisions.
-        pv[0] = new StructuralPropertiesVector(null, null);
-        pv[1] = new StructuralPropertiesVector(new BroadcastPartitioningProperty(context.getComputationNodeDomain()),
-                null);
+        pv[0] = OperatorPropertiesUtil.checkUnpartitionedAndGetPropertiesVector(op, new StructuralPropertiesVector(
+                new RandomPartitioningProperty(context.getComputationNodeDomain()), null));
+        pv[1] = OperatorPropertiesUtil.checkUnpartitionedAndGetPropertiesVector(op, new StructuralPropertiesVector(
+                new BroadcastPartitioningProperty(context.getComputationNodeDomain()), null));
         return new PhysicalRequirements(pv, IPartitioningRequirementsCoordinator.NO_COORDINATION);
     }
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
index d70c67d..d184161 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/operators/physical/UnionAllPOperator.java
@@ -21,20 +21,19 @@ package org.apache.hyracks.algebricks.core.algebra.operators.physical;
 import java.util.ArrayList;
 
 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.IPartitioningProperty.PartitioningType;
+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.RandomPartitioningProperty;
 import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
+import org.apache.hyracks.algebricks.core.algebra.util.OperatorPropertiesUtil;
 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;
@@ -57,17 +56,20 @@ public class UnionAllPOperator extends AbstractPhysicalOperator {
     public void computeDeliveredProperties(ILogicalOperator op, IOptimizationContext context) {
         AbstractLogicalOperator op2 = (AbstractLogicalOperator) op.getInputs().get(0).getValue();
         IPartitioningProperty pp = op2.getDeliveredPhysicalProperties().getPartitioningProperty();
-        this.deliveredProperties = new StructuralPropertiesVector(pp, new ArrayList<ILocalStructuralProperty>(0));
+        this.deliveredProperties = new StructuralPropertiesVector(pp, new ArrayList<>(0));
     }
 
     @Override
     public PhysicalRequirements getRequiredPropertiesForChildren(ILogicalOperator op,
             IPhysicalPropertiesVector reqdByParent, IOptimizationContext context) {
-        StructuralPropertiesVector pv0 = StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
-        StructuralPropertiesVector pv1 = StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR;
+        StructuralPropertiesVector pv0 = OperatorPropertiesUtil.checkUnpartitionedAndGetPropertiesVector(op,
+                new StructuralPropertiesVector(new RandomPartitioningProperty(context.getComputationNodeDomain()),
+                        null));
+        StructuralPropertiesVector pv1 = OperatorPropertiesUtil.checkUnpartitionedAndGetPropertiesVector(op,
+                new StructuralPropertiesVector(new RandomPartitioningProperty(context.getComputationNodeDomain()),
+                        null));
         return new PhysicalRequirements(new StructuralPropertiesVector[] { pv0, pv1 },
-                (requirements, firstDeliveredPartitioning, operator, ctx) -> this.coordinateRequirements(requirements,
-                        firstDeliveredPartitioning));
+                IPartitioningRequirementsCoordinator.NO_COORDINATION);
     }
 
     @Override
@@ -95,20 +97,4 @@ public class UnionAllPOperator extends AbstractPhysicalOperator {
         return false;
     }
 
-    // This method implements how inputs' partitioning properties are coordinated.
-    // The partitioning property of the first input branch is kept unchanged.
-    // A random partitioning property is required for the second branch and the node domain of the first input branch
-    // will be used.
-    private Pair<Boolean, IPartitioningProperty> coordinateRequirements(IPartitioningProperty requirements,
-            IPartitioningProperty firstDeliveredPartitioning) throws AlgebricksException {
-        if (firstDeliveredPartitioning == null) {
-            return new Pair<>(true, requirements);
-        }
-        PartitioningType partType = firstDeliveredPartitioning.getPartitioningType();
-        if (partType == PartitioningType.UNPARTITIONED) {
-            return new Pair<>(true, firstDeliveredPartitioning);
-        } else {
-            return new Pair<>(true, new RandomPartitioningProperty(firstDeliveredPartitioning.getNodeDomain()));
-        }
-    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
index ca101b7..c6b4618 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/properties/StructuralPropertiesVector.java
@@ -86,7 +86,8 @@ public class StructuralPropertiesVector implements IPhysicalPropertiesVector {
         if (reqdPart != null) {
             IPartitioningProperty normalizedReqPart =
                     reqdPart.normalize(equivalenceClasses, mayExpandProperties ? fds : null);
-            IPartitioningProperty normalizedPropPart = propPartitioning.normalize(equivalenceClasses, fds);
+            IPartitioningProperty normalizedPropPart = propPartitioning.normalize(equivalenceClasses,
+                    mayExpandProperties ? fds : null);
             if (!PropertiesUtil.matchPartitioningProps(normalizedReqPart, normalizedPropPart, mayExpandProperties)) {
                 diffPart = reqdPart;
             }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
index 8117921..8b36a68 100644
--- a/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
+++ b/hyracks-fullstack/algebricks/algebricks-core/src/main/java/org/apache/hyracks/algebricks/core/algebra/util/OperatorPropertiesUtil.java
@@ -41,6 +41,7 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AssignOperat
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.SelectOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.CardinalityInferenceVisitor;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.visitors.VariableUtilities;
+import org.apache.hyracks.algebricks.core.algebra.properties.StructuralPropertiesVector;
 
 public class OperatorPropertiesUtil {
 
@@ -330,4 +331,23 @@ public class OperatorPropertiesUtil {
     public static void markMovable(ILogicalOperator op, boolean movable) {
         op.getAnnotations().put(MOVABLE, movable);
     }
+
+    /**
+     * Checks whether a binary input operator in consideration needs to
+     * run in a single partition mode. If it does, returns an empty properties vector.
+     * Otherwise, returns the proposed partitioned properties vector.
+     *
+     * @param op,                          the binary input operator in consideration.
+     * @param partitionedPropertiesVector, the proposed partitioned properties vector.
+     * @return either an empty properties vector or the proposed partitioned properties vector.
+     */
+    public static StructuralPropertiesVector checkUnpartitionedAndGetPropertiesVector(ILogicalOperator op,
+            StructuralPropertiesVector partitionedPropertiesVector) {
+        ILogicalOperator leftChild = op.getInputs().get(0).getValue();
+        ILogicalOperator rightChild = op.getInputs().get(1).getValue();
+        boolean unPartitioned =
+                leftChild.getExecutionMode().equals(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED) && rightChild
+                        .getExecutionMode().equals(AbstractLogicalOperator.ExecutionMode.UNPARTITIONED);
+        return unPartitioned ? StructuralPropertiesVector.EMPTY_PROPERTIES_VECTOR : partitionedPropertiesVector;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java
index 042de2c..3a86565 100644
--- a/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java
+++ b/hyracks-fullstack/algebricks/algebricks-rewriter/src/main/java/org/apache/hyracks/algebricks/rewriter/rules/subplan/PushSubplanIntoGroupByRule.java
@@ -27,7 +27,6 @@ import java.util.Set;
 
 import org.apache.commons.lang3.mutable.Mutable;
 import org.apache.commons.lang3.mutable.MutableObject;
-
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.ListSet;
 import org.apache.hyracks.algebricks.core.algebra.base.ILogicalOperator;
@@ -173,6 +172,8 @@ public class PushSubplanIntoGroupByRule implements IAlgebraicRewriteRule {
         }
         if (changed) {
             cleanup(gby);
+            context.computeAndSetTypeEnvironmentForOperator(gby);
+            context.computeAndSetTypeEnvironmentForOperator(parentOperator);
         }
         return changed;
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
index eb92c37..008a640 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/HyracksConnection.java
@@ -128,13 +128,21 @@ public final class HyracksConnection implements IHyracksClientConnection {
     }
 
     @Override
-    public Map<String, NodeControllerInfo> getNodeControllerInfos() throws Exception {
-        return hci.getNodeControllersInfo();
+    public Map<String, NodeControllerInfo> getNodeControllerInfos() throws HyracksException {
+        try {
+            return hci.getNodeControllersInfo();
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
     }
 
     @Override
-    public ClusterTopology getClusterTopology() throws Exception {
-        return hci.getClusterTopology();
+    public ClusterTopology getClusterTopology() throws HyracksException {
+        try {
+            return hci.getClusterTopology();
+        } catch (Exception e) {
+            throw new HyracksException(e);
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IClusterInfoCollector.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IClusterInfoCollector.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IClusterInfoCollector.java
new file mode 100644
index 0000000..139dc3d
--- /dev/null
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IClusterInfoCollector.java
@@ -0,0 +1,46 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+
+package org.apache.hyracks.api.client;
+
+import java.util.Map;
+
+import org.apache.hyracks.api.exceptions.HyracksException;
+import org.apache.hyracks.api.topology.ClusterTopology;
+
+public interface IClusterInfoCollector {
+
+    /**
+     * Gets a map of node controller names to node information.
+     *
+     * @return Map of node name to node information.
+     * @throws HyracksException
+     */
+    public Map<String, NodeControllerInfo> getNodeControllerInfos() throws HyracksException;
+
+    /**
+     * Get the cluster topology
+     *
+     * @return the cluster topology
+     * @throws HyracksException
+     */
+    public ClusterTopology getClusterTopology() throws HyracksException;
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
index 031896e..c4eba3d 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/IHyracksClientConnection.java
@@ -20,7 +20,6 @@ package org.apache.hyracks.api.client;
 
 import java.util.EnumSet;
 import java.util.List;
-import java.util.Map;
 
 import org.apache.hyracks.api.comm.NetworkAddress;
 import org.apache.hyracks.api.deployment.DeploymentId;
@@ -30,14 +29,13 @@ import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobInfo;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.apache.hyracks.api.job.JobStatus;
-import org.apache.hyracks.api.topology.ClusterTopology;
 
 /**
  * Interface used by clients to communicate with the Hyracks Cluster Controller.
  *
  * @author vinayakb
  */
-public interface IHyracksClientConnection {
+public interface IHyracksClientConnection extends IClusterInfoCollector {
     /**
      * Gets the status of the specified Job.
      *
@@ -61,8 +59,6 @@ public interface IHyracksClientConnection {
     /**
      * Start the specified Job.
      *
-     * @param appName
-     *            Name of the application
      * @param jobSpec
      *            Job Specification
      * @throws Exception
@@ -72,8 +68,6 @@ public interface IHyracksClientConnection {
     /**
      * Start the specified Job.
      *
-     * @param appName
-     *            Name of the application
      * @param jobSpec
      *            Job Specification
      * @param jobFlags
@@ -85,8 +79,6 @@ public interface IHyracksClientConnection {
     /**
      * Start the specified Job.
      *
-     * @param appName
-     *            Name of the application
      * @param jobSpec
      *            Job Specification
      * @param jobFlags
@@ -100,8 +92,6 @@ public interface IHyracksClientConnection {
     /**
      * Start the specified Job.
      *
-     * @param appName
-     *            Name of the application
      * @param acggf
      *            Activity Cluster Graph Generator Factory
      * @param jobFlags
@@ -128,20 +118,6 @@ public interface IHyracksClientConnection {
      */
     public void waitForCompletion(JobId jobId) throws Exception;
 
-    /**
-     * Gets a map of node controller names to node information.
-     *
-     * @return Map of node name to node information.
-     */
-    public Map<String, NodeControllerInfo> getNodeControllerInfos() throws Exception;
-
-    /**
-     * Get the cluster topology
-     *
-     * @return the cluster topology
-     * @throws Exception
-     */
-    public ClusterTopology getClusterTopology() throws Exception;
 
     /**
      * Deploy the user-defined jars to the cluster
@@ -154,8 +130,8 @@ public interface IHyracksClientConnection {
     /**
      * undeploy a certain deployment
      *
-     * @param jars
-     *            a list of user-defined jars
+     * @param deploymentId
+     *            the id for the deployment to be undeployed
      */
     public void unDeployBinary(DeploymentId deploymentId) throws Exception;
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
index a79b955..c90644f 100644
--- a/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
+++ b/hyracks-fullstack/hyracks/hyracks-api/src/main/java/org/apache/hyracks/api/client/NodeControllerInfo.java
@@ -35,13 +35,16 @@ public class NodeControllerInfo implements Serializable {
 
     private final NetworkAddress messagingNetworkAddress;
 
+    private final int numCores;
+
     public NodeControllerInfo(String nodeId, NodeStatus status, NetworkAddress netAddress,
-            NetworkAddress datasetNetworkAddress, NetworkAddress messagingNetworkAddress) {
+            NetworkAddress datasetNetworkAddress, NetworkAddress messagingNetworkAddress, int numCores) {
         this.nodeId = nodeId;
         this.status = status;
         this.netAddress = netAddress;
         this.datasetNetworkAddress = datasetNetworkAddress;
         this.messagingNetworkAddress = messagingNetworkAddress;
+        this.numCores = numCores;
     }
 
     public String getNodeId() {
@@ -63,4 +66,8 @@ public class NodeControllerInfo implements Serializable {
     public NetworkAddress getMessagingNetworkAddress() {
         return messagingNetworkAddress;
     }
+
+    public int getNumCores() {
+        return numCores;
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
index a4d44b1..ca980b9 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/NodeControllerState.java
@@ -141,6 +141,8 @@ public class NodeControllerState {
 
     private int lastHeartbeatDuration;
 
+    private int numCores;
+
     public NodeControllerState(INodeController nodeController, NodeRegistration reg) {
         this.nodeController = nodeController;
         ncConfig = reg.getNCConfig();
@@ -202,6 +204,7 @@ public class NodeControllerState {
         diskWrites = new long[RRD_SIZE];
 
         rrdPtr = 0;
+        numCores = 0;
     }
 
     public synchronized void notifyHeartbeat(HeartbeatData hbData) {
@@ -239,6 +242,7 @@ public class NodeControllerState {
             diskReads[rrdPtr] = hbData.diskReads;
             diskWrites[rrdPtr] = hbData.diskWrites;
             rrdPtr = (rrdPtr + 1) % RRD_SIZE;
+            numCores = hbData.numCores;
         }
     }
 
@@ -274,6 +278,10 @@ public class NodeControllerState {
         return messagingPort;
     }
 
+    public int getNumCores() {
+        return numCores;
+    }
+
     public synchronized JSONObject toSummaryJSON() throws JSONException {
         JSONObject o = new JSONObject();
         o.put("node-id", ncConfig.nodeId);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
index 726bf12..a0150f2 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-cc/src/main/java/org/apache/hyracks/control/cc/work/GetNodeControllersInfoWork.java
@@ -43,8 +43,9 @@ public class GetNodeControllersInfoWork extends AbstractWork {
         Map<String, NodeControllerInfo> result = new LinkedHashMap<>();
         Map<String, NodeControllerState> nodeMap = ccs.getNodeMap();
         for (Map.Entry<String, NodeControllerState> e : nodeMap.entrySet()) {
-            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, e.getValue().getDataPort(),
-                    e.getValue().getDatasetPort(), e.getValue().getMessagingPort()));
+            NodeControllerState ncState = e.getValue();
+            result.put(e.getKey(), new NodeControllerInfo(e.getKey(), NodeStatus.ALIVE, ncState.getDataPort(),
+                    ncState.getDatasetPort(), ncState.getMessagingPort(), ncState.getNumCores()));
         }
         callback.setValue(result);
     }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java
index cfb9a86..1dbb5b6 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-common/src/main/java/org/apache/hyracks/control/common/heartbeat/HeartbeatData.java
@@ -52,6 +52,7 @@ public class HeartbeatData {
     public long ipcMessageBytesReceived;
     public long diskReads;
     public long diskWrites;
+    public int numCores;
 
     public void readFields(DataInput dis) throws IOException {
         heapInitSize = dis.readLong();
@@ -80,6 +81,7 @@ public class HeartbeatData {
         ipcMessageBytesReceived = dis.readLong();
         diskReads = dis.readLong();
         diskWrites = dis.readLong();
+        numCores = dis.readInt();
 
         int gcCounts = dis.readInt();
         gcCollectionCounts = new long[gcCounts];
@@ -120,6 +122,7 @@ public class HeartbeatData {
         dos.writeLong(ipcMessageBytesReceived);
         dos.writeLong(diskReads);
         dos.writeLong(diskWrites);
+        dos.writeInt(numCores);
 
         dos.writeInt(gcCollectionCounts.length);
         for (int i = 0; i < gcCollectionCounts.length; i++) {

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
index bdb13e4..d44a4e5 100644
--- a/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
+++ b/hyracks-fullstack/hyracks/hyracks-control/hyracks-control-nc/src/main/java/org/apache/hyracks/control/nc/NodeControllerService.java
@@ -455,6 +455,7 @@ public class NodeControllerService implements IControllerService {
 
             hbData.diskReads = ioCounter.getReads();
             hbData.diskWrites = ioCounter.getWrites();
+            hbData.numCores = Runtime.getRuntime().availableProcessors();
 
             try {
                 cc.nodeHeartbeat(id, hbData);

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
index 06497d5..f20fe8d 100644
--- a/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
+++ b/hyracks-fullstack/hyracks/hyracks-dataflow-std/src/main/java/org/apache/hyracks/dataflow/std/union/UnionAllOperatorDescriptor.java
@@ -63,21 +63,15 @@ public class UnionAllOperatorDescriptor extends AbstractOperatorDescriptor {
         public IOperatorNodePushable createPushRuntime(IHyracksTaskContext ctx,
                 IRecordDescriptorProvider recordDescProvider, int partition, int nPartitions)
                         throws HyracksDataException {
-            RecordDescriptor inRecordDesc = recordDescProvider.getInputRecordDescriptor(getActivityId(), 0);
-            return new UnionOperator(ctx, inRecordDesc);
+            return new UnionOperator();
         }
     }
 
     private class UnionOperator extends AbstractUnaryOutputOperatorNodePushable {
-        private int nOpened;
-        private int nClosed;
+        private int nOpened = 0;
+        private int nClosed = 0;
         private boolean failed;
 
-        public UnionOperator(IHyracksTaskContext ctx, RecordDescriptor inRecordDesc) {
-            nOpened = 0;
-            nClosed = 0;
-        }
-
         @Override
         public int getInputArity() {
             return inputArity;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/scheduler/SchedulerTest.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/scheduler/SchedulerTest.java b/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/scheduler/SchedulerTest.java
index 13cb171..6e248b5 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/scheduler/SchedulerTest.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/scheduler/SchedulerTest.java
@@ -98,10 +98,10 @@ public class SchedulerTest extends TestCase {
                 dataPort, resultPort, messagingPort);
         ncNameToNcInfos.put("nc7",
                 new NodeControllerInfo("nc7", NodeStatus.ALIVE, new NetworkAddress("10.0.0.7", dataPort),
-                        new NetworkAddress("10.0.0.5", resultPort), new NetworkAddress("10.0.0.5", messagingPort)));
+                        new NetworkAddress("10.0.0.5", resultPort), new NetworkAddress("10.0.0.5", messagingPort), 2));
         ncNameToNcInfos.put("nc12",
                 new NodeControllerInfo("nc12", NodeStatus.ALIVE, new NetworkAddress("10.0.0.12", dataPort),
-                        new NetworkAddress("10.0.0.5", resultPort), new NetworkAddress("10.0.0.5", messagingPort)));
+                        new NetworkAddress("10.0.0.5", resultPort), new NetworkAddress("10.0.0.5", messagingPort), 2));
 
         InputSplit[] fileSplits = new InputSplit[12];
         fileSplits[0] = new FileSplit(new Path("part-1"), 0, 0, new String[] { "10.0.0.1", "10.0.0.2", "10.0.0.3" });

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/utils/TestUtils.java
----------------------------------------------------------------------
diff --git a/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/utils/TestUtils.java b/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/utils/TestUtils.java
index 8755cf3..09e3a35 100644
--- a/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/utils/TestUtils.java
+++ b/hyracks-fullstack/hyracks/hyracks-hdfs/hyracks-hdfs-core/src/test/java/org/apache/hyracks/hdfs/utils/TestUtils.java
@@ -108,7 +108,7 @@ public class TestUtils {
             String ncAddress = addressPrefix + i;
             ncNameToNcInfos.put(ncId,
                     new NodeControllerInfo(ncId, NodeStatus.ALIVE, new NetworkAddress(ncAddress, netPort),
-                            new NetworkAddress(ncAddress, dataPort), new NetworkAddress(ncAddress, messagingPort)));
+                            new NetworkAddress(ncAddress, dataPort), new NetworkAddress(ncAddress, messagingPort), 2));
         }
         return ncNameToNcInfos;
     }


[5/8] asterixdb git commit: Supports flexible degree of parallelism.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.2.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.2.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.2.adm
index 94e383c..74e1bb7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.2.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.2.adm
@@ -1,105 +1,456 @@
+{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15" }
+{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30" }
+{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17" }
+{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23" }
+{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01" }
+{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18" }
+{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25" }
+{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24" }
+{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03" }
+{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02" }
+{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03" }
+{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18" }
+{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02" }
+{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14" }
+{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06" }
 { "partkey": 6, "pid": 1, "shipdate": "1992-04-05" }
 { "partkey": 6, "pid": 2, "shipdate": "1992-04-25" }
 { "partkey": 6, "pid": 3, "shipdate": "1992-04-29" }
+{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11" }
+{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25" }
+{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25" }
+{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15" }
+{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13" }
+{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29" }
+{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30" }
+{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13" }
+{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01" }
 { "partkey": 11, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 11, "pid": 2, "shipdate": "1992-07-20" }
 { "partkey": 11, "pid": 3, "shipdate": "1992-08-03" }
 { "partkey": 12, "pid": 1, "shipdate": "1992-07-04" }
 { "partkey": 12, "pid": 2, "shipdate": "1992-07-17" }
 { "partkey": 12, "pid": 3, "shipdate": "1992-09-02" }
+{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01" }
+{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26" }
+{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04" }
 { "partkey": 14, "pid": 1, "shipdate": "1992-07-17" }
 { "partkey": 14, "pid": 2, "shipdate": "1992-11-30" }
 { "partkey": 14, "pid": 3, "shipdate": "1993-05-10" }
+{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18" }
+{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24" }
+{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14" }
+{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11" }
+{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25" }
+{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17" }
+{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23" }
+{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01" }
+{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06" }
+{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21" }
+{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21" }
+{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19" }
+{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21" }
+{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22" }
+{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15" }
+{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29" }
+{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18" }
 { "partkey": 21, "pid": 1, "shipdate": "1992-07-31" }
 { "partkey": 21, "pid": 2, "shipdate": "1992-09-09" }
 { "partkey": 21, "pid": 3, "shipdate": "1993-01-09" }
+{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21" }
+{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25" }
+{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20" }
 { "partkey": 23, "pid": 1, "shipdate": "1992-04-04" }
 { "partkey": 23, "pid": 2, "shipdate": "1992-06-19" }
 { "partkey": 23, "pid": 3, "shipdate": "1992-06-29" }
+{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06" }
+{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08" }
+{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04" }
+{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23" }
+{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01" }
 { "partkey": 26, "pid": 1, "shipdate": "1992-02-23" }
 { "partkey": 26, "pid": 2, "shipdate": "1992-05-09" }
 { "partkey": 26, "pid": 3, "shipdate": "1993-01-04" }
+{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14" }
+{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17" }
+{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16" }
+{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13" }
+{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04" }
+{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25" }
+{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01" }
+{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25" }
 { "partkey": 30, "pid": 1, "shipdate": "1992-04-10" }
 { "partkey": 30, "pid": 2, "shipdate": "1992-05-18" }
 { "partkey": 30, "pid": 3, "shipdate": "1992-05-21" }
+{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14" }
+{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24" }
+{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29" }
+{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22" }
+{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25" }
+{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07" }
 { "partkey": 33, "pid": 1, "shipdate": "1992-03-22" }
 { "partkey": 33, "pid": 2, "shipdate": "1993-02-17" }
 { "partkey": 33, "pid": 3, "shipdate": "1993-02-21" }
+{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03" }
+{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20" }
+{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23" }
+{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11" }
+{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06" }
+{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26" }
+{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26" }
+{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03" }
+{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06" }
+{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30" }
+{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03" }
+{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31" }
 { "partkey": 38, "pid": 1, "shipdate": "1992-04-06" }
 { "partkey": 38, "pid": 2, "shipdate": "1992-04-15" }
 { "partkey": 38, "pid": 3, "shipdate": "1992-08-27" }
+{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26" }
+{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12" }
+{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15" }
+{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07" }
+{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03" }
+{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13" }
+{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18" }
+{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13" }
+{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23" }
+{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04" }
+{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12" }
+{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30" }
+{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28" }
+{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14" }
+{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11" }
+{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29" }
 { "partkey": 45, "pid": 1, "shipdate": "1992-07-16" }
 { "partkey": 45, "pid": 2, "shipdate": "1993-06-24" }
 { "partkey": 45, "pid": 3, "shipdate": "1993-09-15" }
+{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28" }
+{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08" }
+{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21" }
 { "partkey": 47, "pid": 1, "shipdate": "1992-03-11" }
 { "partkey": 47, "pid": 2, "shipdate": "1993-05-30" }
 { "partkey": 47, "pid": 3, "shipdate": "1993-06-06" }
+{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10" }
+{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03" }
+{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15" }
 { "partkey": 49, "pid": 1, "shipdate": "1992-04-29" }
 { "partkey": 49, "pid": 2, "shipdate": "1992-06-14" }
 { "partkey": 49, "pid": 3, "shipdate": "1992-08-13" }
+{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22" }
+{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31" }
+{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23" }
 { "partkey": 51, "pid": 1, "shipdate": "1992-03-11" }
 { "partkey": 51, "pid": 2, "shipdate": "1992-05-15" }
 { "partkey": 51, "pid": 3, "shipdate": "1992-05-17" }
+{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31" }
+{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03" }
+{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21" }
+{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14" }
+{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22" }
+{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04" }
+{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07" }
+{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01" }
+{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24" }
+{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11" }
+{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17" }
+{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02" }
+{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18" }
+{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06" }
+{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21" }
 { "partkey": 58, "pid": 1, "shipdate": "1992-05-16" }
 { "partkey": 58, "pid": 2, "shipdate": "1992-10-30" }
 { "partkey": 58, "pid": 3, "shipdate": "1993-04-10" }
+{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09" }
+{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17" }
+{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12" }
 { "partkey": 60, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 60, "pid": 2, "shipdate": "1992-07-01" }
 { "partkey": 60, "pid": 3, "shipdate": "1992-07-15" }
+{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14" }
+{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15" }
+{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29" }
+{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26" }
+{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19" }
+{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07" }
+{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15" }
+{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07" }
+{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13" }
+{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14" }
+{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10" }
+{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02" }
+{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14" }
+{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26" }
+{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11" }
+{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10" }
+{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13" }
+{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08" }
+{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03" }
+{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13" }
+{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08" }
+{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22" }
+{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31" }
+{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05" }
+{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01" }
 { "partkey": 70, "pid": 1, "shipdate": "1992-04-06" }
 { "partkey": 70, "pid": 2, "shipdate": "1992-06-11" }
 { "partkey": 70, "pid": 3, "shipdate": "1992-06-25" }
+{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10" }
+{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10" }
+{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28" }
 { "partkey": 72, "pid": 1, "shipdate": "1992-09-16" }
 { "partkey": 72, "pid": 2, "shipdate": "1992-10-02" }
 { "partkey": 72, "pid": 3, "shipdate": "1992-10-17" }
+{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08" }
+{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16" }
+{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02" }
+{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21" }
+{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22" }
+{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21" }
+{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27" }
+{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12" }
+{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19" }
+{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22" }
+{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19" }
+{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12" }
 { "partkey": 77, "pid": 1, "shipdate": "1992-08-18" }
 { "partkey": 77, "pid": 2, "shipdate": "1992-12-23" }
 { "partkey": 77, "pid": 3, "shipdate": "1993-06-19" }
+{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04" }
+{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04" }
+{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06" }
+{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05" }
+{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10" }
+{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08" }
+{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18" }
+{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02" }
+{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07" }
+{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11" }
+{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22" }
+{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30" }
+{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17" }
+{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18" }
+{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11" }
+{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09" }
+{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04" }
+{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21" }
+{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08" }
+{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15" }
+{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20" }
+{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28" }
+{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28" }
+{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27" }
+{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25" }
+{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18" }
+{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01" }
+{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30" }
+{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02" }
+{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06" }
 { "partkey": 88, "pid": 1, "shipdate": "1992-04-24" }
 { "partkey": 88, "pid": 2, "shipdate": "1992-06-26" }
 { "partkey": 88, "pid": 3, "shipdate": "1992-12-18" }
 { "partkey": 89, "pid": 1, "shipdate": "1992-04-18" }
 { "partkey": 89, "pid": 2, "shipdate": "1992-04-19" }
 { "partkey": 89, "pid": 3, "shipdate": "1992-05-27" }
+{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25" }
+{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07" }
+{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21" }
+{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22" }
+{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21" }
+{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03" }
 { "partkey": 92, "pid": 1, "shipdate": "1992-02-11" }
 { "partkey": 92, "pid": 2, "shipdate": "1992-09-30" }
 { "partkey": 92, "pid": 3, "shipdate": "1993-01-04" }
 { "partkey": 93, "pid": 1, "shipdate": "1992-05-28" }
 { "partkey": 93, "pid": 2, "shipdate": "1992-06-24" }
 { "partkey": 93, "pid": 3, "shipdate": "1992-09-11" }
+{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20" }
+{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03" }
+{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26" }
+{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24" }
+{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14" }
+{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17" }
+{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26" }
+{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25" }
+{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27" }
+{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22" }
+{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21" }
+{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06" }
+{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09" }
+{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09" }
+{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01" }
+{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18" }
+{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09" }
+{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24" }
+{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24" }
+{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18" }
+{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17" }
+{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27" }
+{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28" }
+{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19" }
+{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21" }
+{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25" }
 { "partkey": 103, "pid": 1, "shipdate": "1992-03-28" }
 { "partkey": 103, "pid": 2, "shipdate": "1992-05-08" }
 { "partkey": 103, "pid": 3, "shipdate": "1992-07-11" }
+{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17" }
+{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08" }
+{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22" }
 { "partkey": 105, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 105, "pid": 2, "shipdate": "1992-06-01" }
 { "partkey": 105, "pid": 3, "shipdate": "1992-07-14" }
+{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09" }
+{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31" }
+{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02" }
+{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22" }
+{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30" }
+{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05" }
+{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28" }
+{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01" }
+{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07" }
 { "partkey": 109, "pid": 1, "shipdate": "1992-06-06" }
 { "partkey": 109, "pid": 2, "shipdate": "1992-11-20" }
 { "partkey": 109, "pid": 3, "shipdate": "1992-12-23" }
-{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13" }
-{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29" }
-{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17" }
-{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27" }
-{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02" }
-{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15" }
+{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18" }
+{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01" }
+{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01" }
+{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28" }
+{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13" }
+{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13" }
+{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09" }
+{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15" }
+{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08" }
+{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13" }
+{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25" }
+{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19" }
+{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22" }
+{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22" }
+{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13" }
+{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29" }
+{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17" }
+{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22" }
+{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17" }
+{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24" }
+{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04" }
+{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18" }
+{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10" }
+{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27" }
+{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02" }
+{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08" }
+{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27" }
+{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07" }
+{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23" }
+{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29" }
+{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23" }
+{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09" }
+{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23" }
+{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12" }
+{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09" }
+{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05" }
+{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22" }
+{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15" }
+{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09" }
+{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13" }
+{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15" }
 { "partkey": 125, "pid": 2, "shipdate": "1992-03-29" }
 { "partkey": 125, "pid": 3, "shipdate": "1992-05-24" }
 { "partkey": 126, "pid": 1, "shipdate": "1992-07-28" }
 { "partkey": 126, "pid": 2, "shipdate": "1992-08-28" }
 { "partkey": 126, "pid": 3, "shipdate": "1992-09-06" }
+{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04" }
+{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02" }
+{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13" }
+{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05" }
+{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02" }
+{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24" }
+{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31" }
+{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28" }
+{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15" }
+{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03" }
+{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23" }
+{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20" }
+{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27" }
+{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03" }
+{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17" }
+{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14" }
+{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06" }
+{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08" }
+{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17" }
+{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18" }
+{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17" }
+{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20" }
+{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29" }
 { "partkey": 135, "pid": 1, "shipdate": "1992-05-02" }
 { "partkey": 135, "pid": 2, "shipdate": "1992-05-11" }
 { "partkey": 135, "pid": 3, "shipdate": "1992-05-29" }
+{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19" }
+{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21" }
+{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07" }
+{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23" }
+{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05" }
+{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12" }
 { "partkey": 138, "pid": 1, "shipdate": "1992-06-20" }
 { "partkey": 138, "pid": 2, "shipdate": "1992-11-21" }
 { "partkey": 138, "pid": 3, "shipdate": "1993-02-28" }
+{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28" }
+{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12" }
+{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20" }
+{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27" }
+{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03" }
 { "partkey": 141, "pid": 1, "shipdate": "1992-01-13" }
 { "partkey": 141, "pid": 2, "shipdate": "1992-02-01" }
 { "partkey": 141, "pid": 3, "shipdate": "1992-06-22" }
+{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14" }
+{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14" }
+{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11" }
+{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17" }
+{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01" }
+{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05" }
+{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25" }
+{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17" }
+{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25" }
+{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16" }
+{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25" }
+{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21" }
+{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21" }
+{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02" }
 { "partkey": 147, "pid": 1, "shipdate": "1992-06-10" }
 { "partkey": 147, "pid": 2, "shipdate": "1992-09-04" }
 { "partkey": 147, "pid": 3, "shipdate": "1992-12-03" }
+{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15" }
+{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27" }
+{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22" }
+{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22" }
+{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29" }
+{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01" }
+{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02" }
+{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25" }
+{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26" }
+{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30" }
+{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19" }
 { "partkey": 152, "pid": 1, "shipdate": "1992-06-23" }
 { "partkey": 152, "pid": 2, "shipdate": "1993-05-19" }
 { "partkey": 152, "pid": 3, "shipdate": "1993-10-31" }
@@ -109,18 +460,75 @@
 { "partkey": 154, "pid": 1, "shipdate": "1992-02-18" }
 { "partkey": 154, "pid": 2, "shipdate": "1992-02-20" }
 { "partkey": 154, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28" }
+{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14" }
 { "partkey": 156, "pid": 1, "shipdate": "1992-04-24" }
 { "partkey": 156, "pid": 2, "shipdate": "1992-06-17" }
 { "partkey": 156, "pid": 3, "shipdate": "1992-07-01" }
 { "partkey": 157, "pid": 1, "shipdate": "1992-07-26" }
 { "partkey": 157, "pid": 2, "shipdate": "1992-08-11" }
 { "partkey": 157, "pid": 3, "shipdate": "1992-08-25" }
+{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01" }
+{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29" }
+{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18" }
+{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03" }
+{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10" }
+{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04" }
+{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18" }
+{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29" }
+{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18" }
+{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28" }
+{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10" }
+{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03" }
+{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11" }
+{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09" }
+{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27" }
+{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25" }
+{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17" }
+{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06" }
+{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21" }
+{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01" }
+{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12" }
+{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11" }
+{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14" }
+{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22" }
+{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02" }
+{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31" }
+{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15" }
+{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06" }
+{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20" }
+{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07" }
+{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31" }
+{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05" }
+{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07" }
 { "partkey": 170, "pid": 1, "shipdate": "1992-08-07" }
 { "partkey": 170, "pid": 2, "shipdate": "1993-03-17" }
 { "partkey": 170, "pid": 3, "shipdate": "1993-06-19" }
 { "partkey": 171, "pid": 1, "shipdate": "1992-11-09" }
 { "partkey": 171, "pid": 2, "shipdate": "1994-01-22" }
 { "partkey": 171, "pid": 3, "shipdate": "1995-01-02" }
+{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06" }
+{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01" }
+{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16" }
+{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17" }
+{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15" }
+{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30" }
+{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25" }
+{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02" }
+{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02" }
+{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09" }
+{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09" }
+{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10" }
+{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24" }
+{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05" }
+{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25" }
+{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16" }
 { "partkey": 178, "pid": 1, "shipdate": "1992-05-23" }
 { "partkey": 178, "pid": 2, "shipdate": "1992-08-18" }
 { "partkey": 178, "pid": 3, "shipdate": "1992-11-02" }
@@ -130,471 +538,63 @@
 { "partkey": 180, "pid": 1, "shipdate": "1992-03-07" }
 { "partkey": 180, "pid": 2, "shipdate": "1992-05-23" }
 { "partkey": 180, "pid": 3, "shipdate": "1992-06-21" }
+{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01" }
+{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04" }
+{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14" }
 { "partkey": 182, "pid": 1, "shipdate": "1992-03-02" }
 { "partkey": 182, "pid": 2, "shipdate": "1992-04-02" }
 { "partkey": 182, "pid": 3, "shipdate": "1992-04-28" }
+{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24" }
+{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24" }
+{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08" }
+{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12" }
+{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30" }
+{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30" }
+{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23" }
+{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26" }
+{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27" }
+{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01" }
+{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30" }
+{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15" }
+{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08" }
+{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03" }
+{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16" }
+{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20" }
 { "partkey": 190, "pid": 1, "shipdate": "1992-04-14" }
 { "partkey": 190, "pid": 2, "shipdate": "1992-07-17" }
 { "partkey": 190, "pid": 3, "shipdate": "1992-10-12" }
+{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31" }
+{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29" }
+{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22" }
+{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19" }
+{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10" }
+{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02" }
+{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05" }
+{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21" }
+{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12" }
+{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14" }
+{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15" }
 { "partkey": 195, "pid": 1, "shipdate": "1992-04-10" }
 { "partkey": 195, "pid": 2, "shipdate": "1992-05-07" }
 { "partkey": 195, "pid": 3, "shipdate": "1992-05-28" }
+{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02" }
+{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04" }
+{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11" }
+{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22" }
+{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24" }
+{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03" }
+{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21" }
+{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12" }
+{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27" }
 { "partkey": 199, "pid": 1, "shipdate": "1992-03-14" }
 { "partkey": 199, "pid": 2, "shipdate": "1992-08-02" }
 { "partkey": 199, "pid": 3, "shipdate": "1992-11-20" }
-{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15" }
-{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30" }
-{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17" }
-{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23" }
-{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01" }
-{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18" }
-{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02" }
-{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03" }
-{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18" }
-{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01" }
-{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26" }
-{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04" }
-{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18" }
-{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24" }
-{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14" }
-{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11" }
-{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25" }
-{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17" }
-{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19" }
-{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21" }
-{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22" }
-{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15" }
-{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29" }
-{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18" }
-{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21" }
-{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25" }
-{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20" }
-{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06" }
-{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08" }
-{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25" }
-{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01" }
-{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25" }
-{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14" }
-{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24" }
-{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29" }
-{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11" }
-{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06" }
-{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26" }
-{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28" }
-{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08" }
-{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21" }
-{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10" }
-{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03" }
-{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15" }
-{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31" }
-{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03" }
-{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11" }
-{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17" }
-{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02" }
-{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18" }
-{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06" }
-{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09" }
-{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17" }
-{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12" }
-{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26" }
-{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19" }
-{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07" }
-{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15" }
-{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07" }
-{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13" }
-{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14" }
-{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10" }
-{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02" }
-{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14" }
-{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26" }
-{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13" }
-{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08" }
-{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22" }
-{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10" }
-{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10" }
-{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28" }
-{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08" }
-{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16" }
-{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02" }
-{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27" }
-{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12" }
-{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19" }
-{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11" }
-{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22" }
-{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30" }
-{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22" }
-{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21" }
-{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03" }
-{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27" }
-{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22" }
-{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21" }
-{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06" }
-{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09" }
-{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09" }
-{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24" }
-{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24" }
-{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18" }
-{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09" }
-{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31" }
-{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02" }
-{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19" }
-{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22" }
-{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22" }
-{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08" }
-{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27" }
-{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07" }
-{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23" }
-{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09" }
-{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23" }
-{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12" }
-{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09" }
-{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05" }
-{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04" }
-{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02" }
-{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13" }
-{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31" }
-{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28" }
-{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15" }
-{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03" }
-{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23" }
-{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20" }
-{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27" }
-{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03" }
-{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14" }
-{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20" }
-{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27" }
-{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03" }
-{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17" }
-{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01" }
-{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05" }
-{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25" }
-{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17" }
-{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22" }
-{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29" }
-{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14" }
-{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01" }
-{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29" }
-{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18" }
-{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03" }
-{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10" }
-{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29" }
-{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18" }
-{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28" }
-{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10" }
-{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03" }
-{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11" }
-{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09" }
-{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27" }
-{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25" }
-{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17" }
-{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06" }
-{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21" }
-{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01" }
-{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12" }
-{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06" }
-{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20" }
-{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07" }
-{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06" }
-{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01" }
-{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16" }
-{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09" }
-{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09" }
-{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10" }
-{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05" }
-{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25" }
-{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16" }
-{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01" }
-{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04" }
-{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14" }
-{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12" }
-{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30" }
-{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26" }
-{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27" }
-{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16" }
-{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20" }
-{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14" }
-{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15" }
-{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22" }
-{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24" }
-{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03" }
-{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21" }
-{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12" }
-{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27" }
-{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25" }
-{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15" }
-{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13" }
-{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29" }
-{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30" }
-{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13" }
-{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01" }
-{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04" }
-{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23" }
-{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01" }
-{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16" }
-{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13" }
-{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04" }
-{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03" }
-{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20" }
-{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23" }
-{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30" }
-{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03" }
-{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31" }
-{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07" }
-{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03" }
-{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13" }
-{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18" }
-{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13" }
-{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30" }
-{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28" }
-{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11" }
-{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10" }
-{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13" }
-{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08" }
-{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03" }
-{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31" }
-{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05" }
-{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01" }
-{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22" }
-{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19" }
-{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12" }
-{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05" }
-{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10" }
-{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08" }
-{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08" }
-{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15" }
-{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20" }
-{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28" }
-{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28" }
-{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27" }
-{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25" }
-{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18" }
-{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01" }
-{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20" }
-{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03" }
-{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26" }
-{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24" }
-{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14" }
-{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17" }
-{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26" }
-{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25" }
-{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01" }
-{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18" }
-{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09" }
-{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19" }
-{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21" }
-{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25" }
-{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28" }
-{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01" }
-{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07" }
-{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28" }
-{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13" }
-{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08" }
-{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13" }
-{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25" }
-{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22" }
-{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17" }
-{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24" }
-{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15" }
-{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09" }
-{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13" }
-{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05" }
-{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02" }
-{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24" }
-{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17" }
-{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14" }
-{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06" }
-{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08" }
-{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17" }
-{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18" }
-{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17" }
-{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20" }
-{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29" }
-{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19" }
-{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21" }
-{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07" }
-{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28" }
-{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12" }
-{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01" }
-{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02" }
-{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25" }
-{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28" }
-{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14" }
-{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04" }
-{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18" }
-{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11" }
-{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14" }
-{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22" }
-{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02" }
-{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31" }
-{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15" }
-{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25" }
-{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02" }
-{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02" }
-{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24" }
-{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24" }
-{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08" }
-{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30" }
-{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23" }
-{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01" }
-{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30" }
-{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31" }
-{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29" }
-{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22" }
-{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19" }
-{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10" }
-{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02" }
-{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02" }
-{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04" }
-{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11" }
 { "partkey": 200, "pid": 1, "shipdate": "1992-04-19" }
 { "partkey": 200, "pid": 2, "shipdate": "1993-01-06" }
 { "partkey": 200, "pid": 3, "shipdate": "1993-10-17" }
-{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25" }
-{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24" }
-{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03" }
-{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02" }
-{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14" }
-{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11" }
-{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25" }
-{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23" }
-{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01" }
-{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06" }
-{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21" }
-{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21" }
-{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14" }
-{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17" }
-{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22" }
-{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25" }
-{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07" }
-{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26" }
-{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03" }
-{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26" }
-{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12" }
-{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15" }
-{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23" }
-{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04" }
-{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12" }
-{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14" }
-{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11" }
-{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29" }
-{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22" }
-{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31" }
-{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23" }
-{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14" }
-{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22" }
-{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04" }
-{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07" }
-{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01" }
-{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24" }
-{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14" }
-{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15" }
-{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29" }
-{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21" }
-{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22" }
-{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21" }
-{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04" }
-{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04" }
-{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06" }
-{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18" }
-{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02" }
-{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07" }
-{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17" }
-{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18" }
-{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11" }
-{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09" }
-{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04" }
-{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30" }
-{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02" }
-{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25" }
-{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07" }
-{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21" }
-{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17" }
-{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27" }
-{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28" }
-{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17" }
-{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08" }
-{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22" }
-{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22" }
-{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30" }
-{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05" }
-{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18" }
-{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01" }
-{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01" }
-{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13" }
-{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09" }
-{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15" }
-{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04" }
-{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18" }
-{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10" }
-{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23" }
-{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29" }
-{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22" }
-{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23" }
-{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05" }
-{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12" }
-{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14" }
-{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14" }
-{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11" }
-{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25" }
-{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16" }
-{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25" }
-{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21" }
-{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21" }
-{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02" }
-{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15" }
-{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27" }
-{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22" }
-{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26" }
-{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30" }
-{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19" }
-{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31" }
-{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05" }
-{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07" }
-{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17" }
-{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15" }
-{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30" }
-{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24" }
-{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15" }
-{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08" }
-{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03" }
-{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05" }
-{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21" }
-{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12" }


[6/8] asterixdb git commit: Supports flexible degree of parallelism.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.1.adm
index 94e383c..74e1bb7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.1.adm
@@ -1,105 +1,456 @@
+{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15" }
+{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30" }
+{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17" }
+{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23" }
+{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01" }
+{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18" }
+{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25" }
+{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24" }
+{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03" }
+{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02" }
+{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03" }
+{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18" }
+{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02" }
+{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14" }
+{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06" }
 { "partkey": 6, "pid": 1, "shipdate": "1992-04-05" }
 { "partkey": 6, "pid": 2, "shipdate": "1992-04-25" }
 { "partkey": 6, "pid": 3, "shipdate": "1992-04-29" }
+{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11" }
+{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25" }
+{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25" }
+{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15" }
+{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13" }
+{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29" }
+{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30" }
+{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13" }
+{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01" }
 { "partkey": 11, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 11, "pid": 2, "shipdate": "1992-07-20" }
 { "partkey": 11, "pid": 3, "shipdate": "1992-08-03" }
 { "partkey": 12, "pid": 1, "shipdate": "1992-07-04" }
 { "partkey": 12, "pid": 2, "shipdate": "1992-07-17" }
 { "partkey": 12, "pid": 3, "shipdate": "1992-09-02" }
+{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01" }
+{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26" }
+{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04" }
 { "partkey": 14, "pid": 1, "shipdate": "1992-07-17" }
 { "partkey": 14, "pid": 2, "shipdate": "1992-11-30" }
 { "partkey": 14, "pid": 3, "shipdate": "1993-05-10" }
+{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18" }
+{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24" }
+{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14" }
+{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11" }
+{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25" }
+{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17" }
+{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23" }
+{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01" }
+{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06" }
+{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21" }
+{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21" }
+{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19" }
+{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21" }
+{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22" }
+{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15" }
+{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29" }
+{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18" }
 { "partkey": 21, "pid": 1, "shipdate": "1992-07-31" }
 { "partkey": 21, "pid": 2, "shipdate": "1992-09-09" }
 { "partkey": 21, "pid": 3, "shipdate": "1993-01-09" }
+{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21" }
+{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25" }
+{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20" }
 { "partkey": 23, "pid": 1, "shipdate": "1992-04-04" }
 { "partkey": 23, "pid": 2, "shipdate": "1992-06-19" }
 { "partkey": 23, "pid": 3, "shipdate": "1992-06-29" }
+{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06" }
+{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08" }
+{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04" }
+{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23" }
+{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01" }
 { "partkey": 26, "pid": 1, "shipdate": "1992-02-23" }
 { "partkey": 26, "pid": 2, "shipdate": "1992-05-09" }
 { "partkey": 26, "pid": 3, "shipdate": "1993-01-04" }
+{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14" }
+{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17" }
+{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16" }
+{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13" }
+{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04" }
+{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25" }
+{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01" }
+{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25" }
 { "partkey": 30, "pid": 1, "shipdate": "1992-04-10" }
 { "partkey": 30, "pid": 2, "shipdate": "1992-05-18" }
 { "partkey": 30, "pid": 3, "shipdate": "1992-05-21" }
+{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14" }
+{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24" }
+{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29" }
+{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22" }
+{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25" }
+{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07" }
 { "partkey": 33, "pid": 1, "shipdate": "1992-03-22" }
 { "partkey": 33, "pid": 2, "shipdate": "1993-02-17" }
 { "partkey": 33, "pid": 3, "shipdate": "1993-02-21" }
+{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03" }
+{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20" }
+{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23" }
+{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11" }
+{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06" }
+{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26" }
+{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26" }
+{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03" }
+{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06" }
+{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30" }
+{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03" }
+{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31" }
 { "partkey": 38, "pid": 1, "shipdate": "1992-04-06" }
 { "partkey": 38, "pid": 2, "shipdate": "1992-04-15" }
 { "partkey": 38, "pid": 3, "shipdate": "1992-08-27" }
+{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26" }
+{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12" }
+{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15" }
+{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07" }
+{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03" }
+{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13" }
+{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18" }
+{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13" }
+{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23" }
+{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04" }
+{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12" }
+{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30" }
+{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28" }
+{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14" }
+{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11" }
+{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29" }
 { "partkey": 45, "pid": 1, "shipdate": "1992-07-16" }
 { "partkey": 45, "pid": 2, "shipdate": "1993-06-24" }
 { "partkey": 45, "pid": 3, "shipdate": "1993-09-15" }
+{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28" }
+{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08" }
+{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21" }
 { "partkey": 47, "pid": 1, "shipdate": "1992-03-11" }
 { "partkey": 47, "pid": 2, "shipdate": "1993-05-30" }
 { "partkey": 47, "pid": 3, "shipdate": "1993-06-06" }
+{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10" }
+{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03" }
+{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15" }
 { "partkey": 49, "pid": 1, "shipdate": "1992-04-29" }
 { "partkey": 49, "pid": 2, "shipdate": "1992-06-14" }
 { "partkey": 49, "pid": 3, "shipdate": "1992-08-13" }
+{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22" }
+{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31" }
+{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23" }
 { "partkey": 51, "pid": 1, "shipdate": "1992-03-11" }
 { "partkey": 51, "pid": 2, "shipdate": "1992-05-15" }
 { "partkey": 51, "pid": 3, "shipdate": "1992-05-17" }
+{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31" }
+{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03" }
+{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21" }
+{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14" }
+{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22" }
+{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04" }
+{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07" }
+{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01" }
+{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24" }
+{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11" }
+{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17" }
+{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02" }
+{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18" }
+{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06" }
+{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21" }
 { "partkey": 58, "pid": 1, "shipdate": "1992-05-16" }
 { "partkey": 58, "pid": 2, "shipdate": "1992-10-30" }
 { "partkey": 58, "pid": 3, "shipdate": "1993-04-10" }
+{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09" }
+{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17" }
+{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12" }
 { "partkey": 60, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 60, "pid": 2, "shipdate": "1992-07-01" }
 { "partkey": 60, "pid": 3, "shipdate": "1992-07-15" }
+{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14" }
+{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15" }
+{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29" }
+{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26" }
+{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19" }
+{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07" }
+{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15" }
+{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07" }
+{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13" }
+{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14" }
+{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10" }
+{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02" }
+{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14" }
+{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26" }
+{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11" }
+{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10" }
+{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13" }
+{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08" }
+{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03" }
+{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13" }
+{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08" }
+{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22" }
+{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31" }
+{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05" }
+{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01" }
 { "partkey": 70, "pid": 1, "shipdate": "1992-04-06" }
 { "partkey": 70, "pid": 2, "shipdate": "1992-06-11" }
 { "partkey": 70, "pid": 3, "shipdate": "1992-06-25" }
+{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10" }
+{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10" }
+{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28" }
 { "partkey": 72, "pid": 1, "shipdate": "1992-09-16" }
 { "partkey": 72, "pid": 2, "shipdate": "1992-10-02" }
 { "partkey": 72, "pid": 3, "shipdate": "1992-10-17" }
+{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08" }
+{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16" }
+{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02" }
+{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21" }
+{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22" }
+{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21" }
+{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27" }
+{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12" }
+{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19" }
+{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22" }
+{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19" }
+{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12" }
 { "partkey": 77, "pid": 1, "shipdate": "1992-08-18" }
 { "partkey": 77, "pid": 2, "shipdate": "1992-12-23" }
 { "partkey": 77, "pid": 3, "shipdate": "1993-06-19" }
+{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04" }
+{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04" }
+{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06" }
+{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05" }
+{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10" }
+{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08" }
+{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18" }
+{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02" }
+{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07" }
+{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11" }
+{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22" }
+{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30" }
+{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17" }
+{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18" }
+{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11" }
+{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09" }
+{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04" }
+{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21" }
+{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08" }
+{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15" }
+{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20" }
+{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28" }
+{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28" }
+{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27" }
+{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25" }
+{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18" }
+{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01" }
+{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30" }
+{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02" }
+{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06" }
 { "partkey": 88, "pid": 1, "shipdate": "1992-04-24" }
 { "partkey": 88, "pid": 2, "shipdate": "1992-06-26" }
 { "partkey": 88, "pid": 3, "shipdate": "1992-12-18" }
 { "partkey": 89, "pid": 1, "shipdate": "1992-04-18" }
 { "partkey": 89, "pid": 2, "shipdate": "1992-04-19" }
 { "partkey": 89, "pid": 3, "shipdate": "1992-05-27" }
+{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25" }
+{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07" }
+{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21" }
+{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22" }
+{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21" }
+{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03" }
 { "partkey": 92, "pid": 1, "shipdate": "1992-02-11" }
 { "partkey": 92, "pid": 2, "shipdate": "1992-09-30" }
 { "partkey": 92, "pid": 3, "shipdate": "1993-01-04" }
 { "partkey": 93, "pid": 1, "shipdate": "1992-05-28" }
 { "partkey": 93, "pid": 2, "shipdate": "1992-06-24" }
 { "partkey": 93, "pid": 3, "shipdate": "1992-09-11" }
+{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20" }
+{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03" }
+{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26" }
+{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24" }
+{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14" }
+{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17" }
+{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26" }
+{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25" }
+{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27" }
+{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22" }
+{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21" }
+{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06" }
+{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09" }
+{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09" }
+{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01" }
+{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18" }
+{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09" }
+{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24" }
+{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24" }
+{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18" }
+{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17" }
+{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27" }
+{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28" }
+{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19" }
+{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21" }
+{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25" }
 { "partkey": 103, "pid": 1, "shipdate": "1992-03-28" }
 { "partkey": 103, "pid": 2, "shipdate": "1992-05-08" }
 { "partkey": 103, "pid": 3, "shipdate": "1992-07-11" }
+{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17" }
+{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08" }
+{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22" }
 { "partkey": 105, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 105, "pid": 2, "shipdate": "1992-06-01" }
 { "partkey": 105, "pid": 3, "shipdate": "1992-07-14" }
+{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09" }
+{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31" }
+{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02" }
+{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22" }
+{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30" }
+{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05" }
+{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28" }
+{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01" }
+{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07" }
 { "partkey": 109, "pid": 1, "shipdate": "1992-06-06" }
 { "partkey": 109, "pid": 2, "shipdate": "1992-11-20" }
 { "partkey": 109, "pid": 3, "shipdate": "1992-12-23" }
-{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13" }
-{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29" }
-{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17" }
-{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27" }
-{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02" }
-{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15" }
+{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18" }
+{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01" }
+{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01" }
+{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28" }
+{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13" }
+{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13" }
+{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09" }
+{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15" }
+{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08" }
+{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13" }
+{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25" }
+{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19" }
+{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22" }
+{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22" }
+{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13" }
+{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29" }
+{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17" }
+{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22" }
+{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17" }
+{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24" }
+{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04" }
+{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18" }
+{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10" }
+{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27" }
+{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02" }
+{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08" }
+{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27" }
+{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07" }
+{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23" }
+{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29" }
+{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23" }
+{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09" }
+{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23" }
+{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12" }
+{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09" }
+{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05" }
+{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22" }
+{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15" }
+{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09" }
+{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13" }
+{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15" }
 { "partkey": 125, "pid": 2, "shipdate": "1992-03-29" }
 { "partkey": 125, "pid": 3, "shipdate": "1992-05-24" }
 { "partkey": 126, "pid": 1, "shipdate": "1992-07-28" }
 { "partkey": 126, "pid": 2, "shipdate": "1992-08-28" }
 { "partkey": 126, "pid": 3, "shipdate": "1992-09-06" }
+{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04" }
+{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02" }
+{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13" }
+{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05" }
+{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02" }
+{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24" }
+{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31" }
+{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28" }
+{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15" }
+{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03" }
+{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23" }
+{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20" }
+{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27" }
+{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03" }
+{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17" }
+{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14" }
+{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06" }
+{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08" }
+{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17" }
+{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18" }
+{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17" }
+{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20" }
+{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29" }
 { "partkey": 135, "pid": 1, "shipdate": "1992-05-02" }
 { "partkey": 135, "pid": 2, "shipdate": "1992-05-11" }
 { "partkey": 135, "pid": 3, "shipdate": "1992-05-29" }
+{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19" }
+{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21" }
+{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07" }
+{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23" }
+{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05" }
+{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12" }
 { "partkey": 138, "pid": 1, "shipdate": "1992-06-20" }
 { "partkey": 138, "pid": 2, "shipdate": "1992-11-21" }
 { "partkey": 138, "pid": 3, "shipdate": "1993-02-28" }
+{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28" }
+{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12" }
+{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20" }
+{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27" }
+{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03" }
 { "partkey": 141, "pid": 1, "shipdate": "1992-01-13" }
 { "partkey": 141, "pid": 2, "shipdate": "1992-02-01" }
 { "partkey": 141, "pid": 3, "shipdate": "1992-06-22" }
+{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14" }
+{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14" }
+{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11" }
+{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17" }
+{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01" }
+{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05" }
+{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25" }
+{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17" }
+{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25" }
+{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16" }
+{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25" }
+{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21" }
+{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21" }
+{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02" }
 { "partkey": 147, "pid": 1, "shipdate": "1992-06-10" }
 { "partkey": 147, "pid": 2, "shipdate": "1992-09-04" }
 { "partkey": 147, "pid": 3, "shipdate": "1992-12-03" }
+{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15" }
+{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27" }
+{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22" }
+{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22" }
+{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29" }
+{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01" }
+{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02" }
+{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25" }
+{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26" }
+{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30" }
+{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19" }
 { "partkey": 152, "pid": 1, "shipdate": "1992-06-23" }
 { "partkey": 152, "pid": 2, "shipdate": "1993-05-19" }
 { "partkey": 152, "pid": 3, "shipdate": "1993-10-31" }
@@ -109,18 +460,75 @@
 { "partkey": 154, "pid": 1, "shipdate": "1992-02-18" }
 { "partkey": 154, "pid": 2, "shipdate": "1992-02-20" }
 { "partkey": 154, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28" }
+{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14" }
 { "partkey": 156, "pid": 1, "shipdate": "1992-04-24" }
 { "partkey": 156, "pid": 2, "shipdate": "1992-06-17" }
 { "partkey": 156, "pid": 3, "shipdate": "1992-07-01" }
 { "partkey": 157, "pid": 1, "shipdate": "1992-07-26" }
 { "partkey": 157, "pid": 2, "shipdate": "1992-08-11" }
 { "partkey": 157, "pid": 3, "shipdate": "1992-08-25" }
+{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01" }
+{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29" }
+{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18" }
+{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03" }
+{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10" }
+{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04" }
+{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18" }
+{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29" }
+{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18" }
+{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28" }
+{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10" }
+{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03" }
+{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11" }
+{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09" }
+{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27" }
+{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25" }
+{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17" }
+{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06" }
+{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21" }
+{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01" }
+{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12" }
+{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11" }
+{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14" }
+{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22" }
+{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02" }
+{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31" }
+{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15" }
+{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06" }
+{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20" }
+{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07" }
+{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31" }
+{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05" }
+{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07" }
 { "partkey": 170, "pid": 1, "shipdate": "1992-08-07" }
 { "partkey": 170, "pid": 2, "shipdate": "1993-03-17" }
 { "partkey": 170, "pid": 3, "shipdate": "1993-06-19" }
 { "partkey": 171, "pid": 1, "shipdate": "1992-11-09" }
 { "partkey": 171, "pid": 2, "shipdate": "1994-01-22" }
 { "partkey": 171, "pid": 3, "shipdate": "1995-01-02" }
+{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06" }
+{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01" }
+{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16" }
+{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17" }
+{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15" }
+{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30" }
+{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25" }
+{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02" }
+{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02" }
+{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09" }
+{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09" }
+{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10" }
+{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24" }
+{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05" }
+{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25" }
+{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16" }
 { "partkey": 178, "pid": 1, "shipdate": "1992-05-23" }
 { "partkey": 178, "pid": 2, "shipdate": "1992-08-18" }
 { "partkey": 178, "pid": 3, "shipdate": "1992-11-02" }
@@ -130,471 +538,63 @@
 { "partkey": 180, "pid": 1, "shipdate": "1992-03-07" }
 { "partkey": 180, "pid": 2, "shipdate": "1992-05-23" }
 { "partkey": 180, "pid": 3, "shipdate": "1992-06-21" }
+{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01" }
+{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04" }
+{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14" }
 { "partkey": 182, "pid": 1, "shipdate": "1992-03-02" }
 { "partkey": 182, "pid": 2, "shipdate": "1992-04-02" }
 { "partkey": 182, "pid": 3, "shipdate": "1992-04-28" }
+{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24" }
+{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24" }
+{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08" }
+{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12" }
+{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30" }
+{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30" }
+{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23" }
+{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26" }
+{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27" }
+{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01" }
+{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30" }
+{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15" }
+{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08" }
+{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03" }
+{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16" }
+{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20" }
 { "partkey": 190, "pid": 1, "shipdate": "1992-04-14" }
 { "partkey": 190, "pid": 2, "shipdate": "1992-07-17" }
 { "partkey": 190, "pid": 3, "shipdate": "1992-10-12" }
+{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31" }
+{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29" }
+{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22" }
+{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19" }
+{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10" }
+{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02" }
+{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05" }
+{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21" }
+{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12" }
+{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14" }
+{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15" }
 { "partkey": 195, "pid": 1, "shipdate": "1992-04-10" }
 { "partkey": 195, "pid": 2, "shipdate": "1992-05-07" }
 { "partkey": 195, "pid": 3, "shipdate": "1992-05-28" }
+{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02" }
+{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04" }
+{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11" }
+{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22" }
+{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24" }
+{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03" }
+{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21" }
+{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12" }
+{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27" }
 { "partkey": 199, "pid": 1, "shipdate": "1992-03-14" }
 { "partkey": 199, "pid": 2, "shipdate": "1992-08-02" }
 { "partkey": 199, "pid": 3, "shipdate": "1992-11-20" }
-{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15" }
-{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30" }
-{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17" }
-{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23" }
-{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01" }
-{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18" }
-{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02" }
-{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03" }
-{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18" }
-{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01" }
-{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26" }
-{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04" }
-{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18" }
-{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24" }
-{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14" }
-{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11" }
-{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25" }
-{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17" }
-{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19" }
-{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21" }
-{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22" }
-{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15" }
-{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29" }
-{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18" }
-{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21" }
-{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25" }
-{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20" }
-{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06" }
-{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08" }
-{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25" }
-{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01" }
-{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25" }
-{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14" }
-{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24" }
-{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29" }
-{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11" }
-{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06" }
-{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26" }
-{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28" }
-{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08" }
-{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21" }
-{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10" }
-{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03" }
-{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15" }
-{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31" }
-{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03" }
-{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11" }
-{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17" }
-{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02" }
-{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18" }
-{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06" }
-{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09" }
-{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17" }
-{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12" }
-{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26" }
-{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19" }
-{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07" }
-{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15" }
-{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07" }
-{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13" }
-{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14" }
-{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10" }
-{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02" }
-{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14" }
-{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26" }
-{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13" }
-{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08" }
-{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22" }
-{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10" }
-{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10" }
-{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28" }
-{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08" }
-{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16" }
-{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02" }
-{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27" }
-{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12" }
-{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19" }
-{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11" }
-{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22" }
-{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30" }
-{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22" }
-{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21" }
-{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03" }
-{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27" }
-{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22" }
-{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21" }
-{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06" }
-{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09" }
-{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09" }
-{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24" }
-{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24" }
-{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18" }
-{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09" }
-{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31" }
-{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02" }
-{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19" }
-{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22" }
-{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22" }
-{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08" }
-{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27" }
-{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07" }
-{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23" }
-{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09" }
-{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23" }
-{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12" }
-{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09" }
-{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05" }
-{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04" }
-{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02" }
-{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13" }
-{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31" }
-{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28" }
-{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15" }
-{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03" }
-{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23" }
-{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20" }
-{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27" }
-{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03" }
-{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14" }
-{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20" }
-{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27" }
-{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03" }
-{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17" }
-{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01" }
-{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05" }
-{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25" }
-{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17" }
-{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22" }
-{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29" }
-{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14" }
-{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01" }
-{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29" }
-{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18" }
-{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03" }
-{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10" }
-{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29" }
-{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18" }
-{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28" }
-{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10" }
-{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03" }
-{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11" }
-{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09" }
-{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27" }
-{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25" }
-{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17" }
-{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06" }
-{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21" }
-{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01" }
-{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12" }
-{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06" }
-{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20" }
-{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07" }
-{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06" }
-{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01" }
-{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16" }
-{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09" }
-{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09" }
-{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10" }
-{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05" }
-{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25" }
-{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16" }
-{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01" }
-{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04" }
-{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14" }
-{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12" }
-{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30" }
-{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26" }
-{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27" }
-{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16" }
-{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20" }
-{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14" }
-{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15" }
-{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22" }
-{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24" }
-{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03" }
-{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21" }
-{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12" }
-{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27" }
-{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25" }
-{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15" }
-{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13" }
-{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29" }
-{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30" }
-{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13" }
-{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01" }
-{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04" }
-{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23" }
-{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01" }
-{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16" }
-{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13" }
-{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04" }
-{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03" }
-{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20" }
-{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23" }
-{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30" }
-{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03" }
-{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31" }
-{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07" }
-{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03" }
-{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13" }
-{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18" }
-{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13" }
-{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30" }
-{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28" }
-{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11" }
-{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10" }
-{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13" }
-{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08" }
-{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03" }
-{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31" }
-{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05" }
-{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01" }
-{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22" }
-{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19" }
-{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12" }
-{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05" }
-{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10" }
-{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08" }
-{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08" }
-{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15" }
-{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20" }
-{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28" }
-{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28" }
-{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27" }
-{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25" }
-{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18" }
-{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01" }
-{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20" }
-{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03" }
-{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26" }
-{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24" }
-{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14" }
-{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17" }
-{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26" }
-{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25" }
-{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01" }
-{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18" }
-{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09" }
-{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19" }
-{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21" }
-{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25" }
-{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28" }
-{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01" }
-{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07" }
-{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28" }
-{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13" }
-{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08" }
-{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13" }
-{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25" }
-{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22" }
-{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17" }
-{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24" }
-{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15" }
-{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09" }
-{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13" }
-{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05" }
-{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02" }
-{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24" }
-{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17" }
-{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14" }
-{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06" }
-{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08" }
-{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17" }
-{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18" }
-{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17" }
-{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20" }
-{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29" }
-{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19" }
-{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21" }
-{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07" }
-{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28" }
-{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12" }
-{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01" }
-{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02" }
-{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25" }
-{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28" }
-{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14" }
-{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04" }
-{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18" }
-{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11" }
-{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14" }
-{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22" }
-{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02" }
-{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31" }
-{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15" }
-{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25" }
-{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02" }
-{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02" }
-{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24" }
-{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24" }
-{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08" }
-{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30" }
-{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23" }
-{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01" }
-{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30" }
-{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31" }
-{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29" }
-{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22" }
-{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19" }
-{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10" }
-{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02" }
-{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02" }
-{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04" }
-{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11" }
 { "partkey": 200, "pid": 1, "shipdate": "1992-04-19" }
 { "partkey": 200, "pid": 2, "shipdate": "1993-01-06" }
 { "partkey": 200, "pid": 3, "shipdate": "1993-10-17" }
-{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25" }
-{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24" }
-{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03" }
-{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02" }
-{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14" }
-{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11" }
-{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25" }
-{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23" }
-{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01" }
-{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06" }
-{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21" }
-{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21" }
-{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14" }
-{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17" }
-{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22" }
-{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25" }
-{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07" }
-{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26" }
-{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03" }
-{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26" }
-{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12" }
-{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15" }
-{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23" }
-{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04" }
-{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12" }
-{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14" }
-{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11" }
-{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29" }
-{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22" }
-{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31" }
-{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23" }
-{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14" }
-{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22" }
-{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04" }
-{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07" }
-{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01" }
-{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24" }
-{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14" }
-{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15" }
-{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29" }
-{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21" }
-{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22" }
-{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21" }
-{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04" }
-{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04" }
-{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06" }
-{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18" }
-{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02" }
-{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07" }
-{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17" }
-{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18" }
-{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11" }
-{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09" }
-{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04" }
-{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30" }
-{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02" }
-{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25" }
-{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07" }
-{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21" }
-{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17" }
-{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27" }
-{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28" }
-{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17" }
-{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08" }
-{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22" }
-{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22" }
-{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30" }
-{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05" }
-{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18" }
-{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01" }
-{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01" }
-{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13" }
-{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09" }
-{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15" }
-{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04" }
-{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18" }
-{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10" }
-{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23" }
-{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29" }
-{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22" }
-{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23" }
-{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05" }
-{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12" }
-{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14" }
-{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14" }
-{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11" }
-{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25" }
-{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16" }
-{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25" }
-{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21" }
-{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21" }
-{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02" }
-{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15" }
-{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27" }
-{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22" }
-{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26" }
-{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30" }
-{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19" }
-{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31" }
-{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05" }
-{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07" }
-{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17" }
-{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15" }
-{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30" }
-{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24" }
-{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15" }
-{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08" }
-{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03" }
-{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05" }
-{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21" }
-{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12" }


[7/8] asterixdb git commit: Supports flexible degree of parallelism.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionTest.java
index 3aae040..f19ebbf 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionTest.java
@@ -18,17 +18,9 @@
  */
 package org.apache.asterix.test.runtime;
 
-import java.io.File;
-import java.util.ArrayList;
 import java.util.Collection;
-import java.util.List;
-import java.util.logging.Logger;
 
-import org.apache.asterix.common.config.TransactionProperties;
-import org.apache.asterix.test.aql.TestExecutor;
 import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.asterix.testframework.xml.TestGroup;
-import org.apache.commons.lang3.StringUtils;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 import org.junit.Test;
@@ -37,62 +29,25 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 /**
- * Runs the runtime test cases under 'asterix-app/src/test/resources/runtimets'.
+ * Runs the SQL++ runtime tests with the storage parallelism.
  */
 @RunWith(Parameterized.class)
 public class SqlppExecutionTest {
-
-    protected static final Logger LOGGER = Logger.getLogger(SqlppExecutionTest.class.getName());
-
-    protected static final String PATH_ACTUAL = "target" + File.separator + "rttest" + File.separator;
-    protected static final String PATH_BASE = StringUtils.join(new String[] { "src", "test", "resources", "runtimets" },
-            File.separator);
-
     protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
 
-    protected static TransactionProperties txnProperties;
-    protected static final List<String> badTestCases = new ArrayList<>();
-    private static final TestExecutor testExecutor = new TestExecutor();
-    private static final boolean cleanupOnStart = true;
-    private static final boolean cleanupOnStop = true;
-
-    protected static TestGroup FailedGroup;
-
     @BeforeClass
     public static void setUp() throws Exception {
-        File outdir = new File(PATH_ACTUAL);
-        outdir.mkdirs();
-        ExecutionTestUtil.setUp(cleanupOnStart);
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
     }
 
     @AfterClass
     public static void tearDown() throws Exception {
-        ExecutionTestUtil.tearDown(cleanupOnStop);
-        ExecutionTestUtil.integrationUtil.removeTestStorageFiles();
-        if (!badTestCases.isEmpty()) {
-            System.out.println("The following test cases left some data");
-            for (String testCase : badTestCases) {
-                System.out.println(testCase);
-            }
-        }
+        LangExecutionUtil.tearDown();
     }
 
     @Parameters(name = "SqlppExecutionTest {index}: {0}")
     public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> testArgs = buildTestsInXml("only_sqlpp.xml");
-        if (testArgs.size() == 0) {
-            testArgs = buildTestsInXml("testsuite_sqlpp.xml");
-        }
-        return testArgs;
-    }
-
-    protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
-        Collection<Object[]> testArgs = new ArrayList<Object[]>();
-        TestCaseContext.Builder b = new TestCaseContext.Builder();
-        for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
-            testArgs.add(new Object[] { ctx });
-        }
-        return testArgs;
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_sqlpp.xml");
     }
 
     protected TestCaseContext tcCtx;
@@ -103,7 +58,6 @@ public class SqlppExecutionTest {
 
     @Test
     public void test() throws Exception {
-        testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false, FailedGroup);
-        testExecutor.cleanup(tcCtx.toString(), badTestCases);
+        LangExecutionUtil.test(tcCtx);
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/query-ASTERIXDB-1608.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/query-ASTERIXDB-1608.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/query-ASTERIXDB-1608.plan
index 1864e29..264487a 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/query-ASTERIXDB-1608.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/nonpure/query-ASTERIXDB-1608.plan
@@ -1,13 +1,13 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- NESTED_LOOP  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+          -- NESTED_LOOP  |UNPARTITIONED|
             -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
               -- UNNEST  |UNPARTITIONED|
                 -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
-            -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
               -- STREAM_PROJECT  |UNPARTITIONED|
                 -- UNNEST  |UNPARTITIONED|
-                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|
\ No newline at end of file
+                  -- EMPTY_TUPLE_SOURCE  |UNPARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue827-2.plan
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue827-2.plan b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue827-2.plan
index 634f3ea..0c8420c 100644
--- a/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue827-2.plan
+++ b/asterixdb/asterix-app/src/test/resources/optimizerts/results/query-issue827-2.plan
@@ -1,19 +1,19 @@
--- DISTRIBUTE_RESULT  |PARTITIONED|
-  -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-    -- STREAM_PROJECT  |PARTITIONED|
-      -- ASSIGN  |PARTITIONED|
-        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-          -- NESTED_LOOP  |PARTITIONED|
-            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-              -- NESTED_LOOP  |PARTITIONED|
-                -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                  -- NESTED_LOOP  |PARTITIONED|
-                    -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                      -- NESTED_LOOP  |PARTITIONED|
-                        -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                          -- NESTED_LOOP  |PARTITIONED|
-                            -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
-                              -- NESTED_LOOP  |PARTITIONED|
+-- DISTRIBUTE_RESULT  |UNPARTITIONED|
+  -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+    -- STREAM_PROJECT  |UNPARTITIONED|
+      -- ASSIGN  |UNPARTITIONED|
+        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+          -- NESTED_LOOP  |UNPARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+              -- NESTED_LOOP  |UNPARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                  -- NESTED_LOOP  |UNPARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                      -- NESTED_LOOP  |UNPARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                          -- NESTED_LOOP  |UNPARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
+                              -- NESTED_LOOP  |UNPARTITIONED|
                                 -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                   -- AGGREGATE  |UNPARTITIONED|
                                     -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
@@ -29,7 +29,7 @@
                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                                   -- AGGREGATE  |UNPARTITIONED|
                                     -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                                       -- AGGREGATE  |PARTITIONED|
@@ -48,7 +48,7 @@
                                                                 -- DATASOURCE_SCAN  |PARTITIONED|
                                                                   -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                                     -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                            -- BROADCAST_EXCHANGE  |PARTITIONED|
+                            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                               -- AGGREGATE  |UNPARTITIONED|
                                 -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                                   -- AGGREGATE  |PARTITIONED|
@@ -64,7 +64,7 @@
                                                       -- DATASOURCE_SCAN  |PARTITIONED|
                                                         -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                           -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                        -- BROADCAST_EXCHANGE  |PARTITIONED|
+                        -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                           -- AGGREGATE  |UNPARTITIONED|
                             -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                               -- AGGREGATE  |PARTITIONED|
@@ -80,7 +80,7 @@
                                                   -- DATASOURCE_SCAN  |PARTITIONED|
                                                     -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                       -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                    -- BROADCAST_EXCHANGE  |PARTITIONED|
+                    -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                       -- AGGREGATE  |UNPARTITIONED|
                         -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                           -- AGGREGATE  |PARTITIONED|
@@ -101,7 +101,7 @@
                                                         -- DATASOURCE_SCAN  |PARTITIONED|
                                                           -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                                             -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-                -- BROADCAST_EXCHANGE  |PARTITIONED|
+                -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
                   -- AGGREGATE  |UNPARTITIONED|
                     -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                       -- AGGREGATE  |PARTITIONED|
@@ -117,7 +117,7 @@
                                           -- DATASOURCE_SCAN  |PARTITIONED|
                                             -- ONE_TO_ONE_EXCHANGE  |PARTITIONED|
                                               -- EMPTY_TUPLE_SOURCE  |PARTITIONED|
-            -- BROADCAST_EXCHANGE  |PARTITIONED|
+            -- ONE_TO_ONE_EXCHANGE  |UNPARTITIONED|
               -- AGGREGATE  |UNPARTITIONED|
                 -- RANDOM_MERGE_EXCHANGE  |PARTITIONED|
                   -- AGGREGATE  |PARTITIONED|

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state.xml b/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state.xml
new file mode 100644
index 0000000..a9b83cc
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries">
+  <test-group name="api">
+    <test-case FilePath="api">
+      <compilation-unit name="cluster_state_1">
+        <output-dir compare="Text">cluster_state_1</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_full.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_full.xml b/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_full.xml
new file mode 100644
index 0000000..f3ac62e
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_full.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries">
+  <test-group name="api">
+    <test-case FilePath="api">
+      <compilation-unit name="cluster_state_1">
+        <output-dir compare="Text">cluster_state_1_full</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_less.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_less.xml b/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_less.xml
new file mode 100644
index 0000000..ce9c58f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/cluster_state_less.xml
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries">
+  <test-group name="api">
+    <test-case FilePath="api">
+      <compilation-unit name="cluster_state_1">
+        <output-dir compare="Text">cluster_state_1_less</output-dir>
+      </compilation-unit>
+    </test-case>
+  </test-group>
+</test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/only_cluster_state.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/only_cluster_state.xml b/asterixdb/asterix-app/src/test/resources/runtimets/only_cluster_state.xml
new file mode 100644
index 0000000..eea32a7
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/only_cluster_state.xml
@@ -0,0 +1,25 @@
+<?xml version="1.0" encoding="UTF-8" standalone="yes"?>
+<!--
+  ~ /*
+  ~  * Licensed to the Apache Software Foundation (ASF) under one
+  ~  * or more contributor license agreements.  See the NOTICE file
+  ~  * distributed with this work for additional information
+  ~  * regarding copyright ownership.  The ASF licenses this file
+  ~  * to you under the Apache License, Version 2.0 (the
+  ~  * "License"); you may not use this file except in compliance
+  ~  * with the License.  You may obtain a copy of the License at
+  ~  *
+  ~  *   http://www.apache.org/licenses/LICENSE-2.0
+  ~  *
+  ~  * Unless required by applicable law or agreed to in writing,
+  ~  * software distributed under the License is distributed on an
+  ~  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+  ~  * KIND, either express or implied.  See the License for the
+  ~  * specific language governing permissions and limitations
+  ~  * under the License.
+  ~  */
+  -->
+<test-suite xmlns="urn:xml.testframework.asterix.apache.org" ResultOffsetPath="results" QueryOffsetPath="queries">
+  <test-group name="failed">
+  </test-group>
+</test-suite>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/APIQueries.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/APIQueries.xml b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/APIQueries.xml
index 30dea93..eb505f1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/APIQueries.xml
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/api/APIQueries.xml
@@ -18,11 +18,6 @@
  !-->
 <test-group name="api">
   <test-case FilePath="api">
-    <compilation-unit name="cluster_state_1">
-      <output-dir compare="Text">cluster_state_1</output-dir>
-    </compilation-unit>
-  </test-case>
-  <test-case FilePath="api">
     <compilation-unit name="cluster_state_2">
       <output-dir compare="Text">cluster_state_2</output-dir>
       <expected-error>HTTP/1.1 404 Not Found</expected-error>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.3.query.aql
index 9908ab7..29f8eab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.3.query.aql
@@ -25,8 +25,8 @@
 use dataverse test;
 
 for $i in dataset LineItem
-order by $i.l_partkey, $i.l_shipdate
 group by $partkey := $i.l_partkey with $i
-for $j at $p in $i
+for $j at $p in ( for $x in $i order by $x.l_shipdate return $x)
 where $p < 4
-return { "partkey": $partkey, "pid": $p, "shipdate": $j.l_shipdate } 
+order by $partkey
+return { "partkey": $partkey, "pid": $p, "shipdate": $j.l_shipdate }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.4.asyncdefer.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.4.asyncdefer.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.4.asyncdefer.aql
index 1661118..29f8eab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.4.asyncdefer.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.4.asyncdefer.aql
@@ -25,8 +25,8 @@
 use dataverse test;
 
 for $i in dataset LineItem
-order by $i.l_partkey, $i.l_shipdate
 group by $partkey := $i.l_partkey with $i
-for $j at $p in $i
+for $j at $p in ( for $x in $i order by $x.l_shipdate return $x)
 where $p < 4
+order by $partkey
 return { "partkey": $partkey, "pid": $p, "shipdate": $j.l_shipdate }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.async.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.async.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.async.aql
index 1661118..29f8eab 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.async.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at00/at00.5.async.aql
@@ -25,8 +25,8 @@
 use dataverse test;
 
 for $i in dataset LineItem
-order by $i.l_partkey, $i.l_shipdate
 group by $partkey := $i.l_partkey with $i
-for $j at $p in $i
+for $j at $p in ( for $x in $i order by $x.l_shipdate return $x)
 where $p < 4
+order by $partkey
 return { "partkey": $partkey, "pid": $p, "shipdate": $j.l_shipdate }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at06/at06.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at06/at06.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at06/at06.3.query.aql
index 4215ed0..ecac052 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at06/at06.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/flwor/at06/at06.3.query.aql
@@ -25,8 +25,8 @@
 use dataverse test;
 
 for $i in dataset LineItem
-order by $i.l_partkey
 group by $partkey := $i.l_partkey with $i
-for $j at $p in (for $ii in $i order by $ii.l_shipdate return $ii)
+for $j at $p in (for $ii in $i order by $ii.l_shipdate, $ii.l_orderkey return $ii)
 where $p < 4
+order by $partkey, $j.l_shipdate, $j.l_orderkey
 return { "partkey": $partkey, "pid": $p, "shipdate": $j.l_shipdate, "orderkey": $j.l_orderkey }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql
index b0de8da..e123ff8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/open-closed/query-issue456/query-issue456.3.query.aql
@@ -17,7 +17,7 @@
  * under the License.
  */
 /*
- * Description  : This test case is to verify the fix for issue456: 
+ * Description  : This test case is to verify the fix for issue456:
  *                https://code.google.com/p/asterixdb/issues/detail?id=456
  * Expected Res : SUCCESS
  * Date         : 3rd June 2013
@@ -28,4 +28,5 @@ use dataverse test;
 for $x in dataset DataOpen
 let $id := $x.id
 group by $m := $x.int_m with $id
+order by $m
 return [ $m, count($id) ]

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue601/query-issue601.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue601/query-issue601.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue601/query-issue601.3.query.aql
index 5c7bae8..3c4018c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue601/query-issue601.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue601/query-issue601.3.query.aql
@@ -27,6 +27,7 @@ use dataverse tpch;
 
 from $l in dataset('LineItem')
 group by $l_linenumber := $l.l_linenumber keeping $l
+order by $l_linenumber
 select {
   "l_linenumber": $l_linenumber,
   "count_order": count($l)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.aql
index 3d0ceb4..d0213ca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.aql
@@ -49,6 +49,7 @@ select {
 
 from $x in $X
 group by $nation_key := $x.nation_key keeping $x
+order by $nation_key
 select {
     "nation_key": $nation_key,
     "sum_price": from $y in $x

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785/query-issue785.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785/query-issue785.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785/query-issue785.3.query.aql
index 4a46eb3..e5a83c7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785/query-issue785.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch-sql-like/query-issue785/query-issue785.3.query.aql
@@ -39,6 +39,7 @@ from $x in (
   }
 )
 group by $nation_key := $x.nation_key keeping $x
+order by $nation_key
 select {
      "nation_key": $nation_key,
      "sum_price": from $i in $x

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue601/query-issue601.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue601/query-issue601.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue601/query-issue601.3.query.aql
index 54df0c5..9e4cce7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue601/query-issue601.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue601/query-issue601.3.query.aql
@@ -27,6 +27,7 @@ use dataverse tpch;
 
 for $l in dataset('LineItem')
 group by $l_linenumber := $l.l_linenumber with $l
+order by $l_linenumber
 return {
   "l_linenumber": $l_linenumber,
   "count_order": count($l)

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785-2/query-issue785-2.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785-2/query-issue785-2.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785-2/query-issue785-2.3.query.aql
index 4e0784d..f1fe465 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785-2/query-issue785-2.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785-2/query-issue785-2.3.query.aql
@@ -44,11 +44,12 @@ let $sum := sum(for $o in $order return $o.o_totalprice)
 return {
     "nation_key": $nation_key,
     "order_date": $orderdate,
-    "sum_price": $sum 
+    "sum_price": $sum
 })
 
 for $x in $X
 group by $nation_key := $x.nation_key with $x
+order by $nation_key
 return {
     "nation_key": $nation_key,
     "sum_price": for $y in $x

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785/query-issue785.3.query.aql
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785/query-issue785.3.query.aql b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785/query-issue785.3.query.aql
index d0d8b9e..9ad6869 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785/query-issue785.3.query.aql
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries/tpch/query-issue785/query-issue785.3.query.aql
@@ -39,6 +39,7 @@ for $x in (
   }
 )
 group by $nation_key := $x.nation_key with $x
+order by $nation_key
 return {
      "nation_key": $nation_key,
      "sum_price": for $i in $x

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.3.query.sqlpp
index 752f607..cb47163 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.3.query.sqlpp
@@ -27,9 +27,10 @@ use test;
 
 select element {'partkey':gen0.partkey,'pid':p,'shipdate':j.l_shipdate}
 from
-    ( select element {'partkey':partkey, 'i':i}
-      from  (select element l from LineItem l order by l.l_partkey, l.l_shipdate) as i
-      group by i.l_partkey as partkey
+    ( select element {'partkey':i.l_partkey, 'i': (select value x.i from g as x order by x.i.l_shipdate) }
+      from  LineItem as i
+      group by i.l_partkey group as g
     ) as gen0,
     gen0.i as j at p
-where p < 4;
+where p < 4
+order by partkey, shipdate;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.4.asyncdefer.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.4.asyncdefer.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.4.asyncdefer.sqlpp
index aa145ff..7bae0d4 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.4.asyncdefer.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.4.asyncdefer.sqlpp
@@ -27,11 +27,12 @@ use test;
 
 select element {'partkey':gen0.partkey,'pid':p,'shipdate':j.l_shipdate}
 from
-    ( select element {'partkey':partkey, 'i':i}
-      from  (select element l from LineItem l order by l.l_partkey, l.l_shipdate) as i
-      group by i.l_partkey as partkey
+    ( select element {'partkey':i.l_partkey, 'i': (select value x.i from g as x order by x.i.l_shipdate) }
+      from  LineItem as i
+      group by i.l_partkey group as g
     ) as gen0,
     gen0.i as j at p
-where p < 4;
+where p < 4
+order by partkey, shipdate;
 
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.5.async.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.5.async.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.5.async.sqlpp
index 710ea9f..cb47163 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.5.async.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at00/at00.5.async.sqlpp
@@ -27,10 +27,10 @@ use test;
 
 select element {'partkey':gen0.partkey,'pid':p,'shipdate':j.l_shipdate}
 from
-    ( select element {'partkey':partkey, 'i':i}
-      from  (select element l from LineItem l order by l.l_partkey, l.l_shipdate) as i
-      group by i.l_partkey as partkey
+    ( select element {'partkey':i.l_partkey, 'i': (select value x.i from g as x order by x.i.l_shipdate) }
+      from  LineItem as i
+      group by i.l_partkey group as g
     ) as gen0,
     gen0.i as j at p
-where p < 4;
-
+where p < 4
+order by partkey, shipdate;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at06/at06.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at06/at06.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at06/at06.3.query.sqlpp
index aace783..1f4c1c1 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at06/at06.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/flwor/at06/at06.3.query.sqlpp
@@ -27,14 +27,15 @@ use test;
 
 select element {'partkey':gen0.partkey,'pid':p,'shipdate':j.l_shipdate,'orderkey':j.l_orderkey}
 from
-    ( select element {'i':i,'partkey':partkey}
-      from   (select element l from LineItem l order by l.l_partkey) as i
-      group by i.l_partkey as partkey
+    ( select element {'i': (select value x.i from g as x),'partkey':i.l_partkey}
+      from  LineItem as i
+      group by i.l_partkey group as g
     ) as gen0,
     (
-  select element ii
-  from  gen0.i as ii
-  order by ii.l_shipdate
-) as j at p
+     select element ii
+     from  gen0.i as ii
+     order by ii.l_shipdate, ii.l_orderkey
+    ) as j at p
 where (p < 4)
+order by partkey, shipdate, orderkey
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
index 04b753e..5eb9a68 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/open-closed/query-issue456/query-issue456.3.query.sqlpp
@@ -26,8 +26,9 @@
 use test;
 
 
-select element [m,test.count(id)]
+select element [x.int_m,test.count(id)]
 from  DataOpen as x
 with  id as x.id
-group by x.int_m as m
+group by x.int_m
+order by x.int_m
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/group-by.26.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/group-by.26.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/group-by.26.query.sqlpp
index f894340..4132bf3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/group-by.26.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tinysocial/tinysocial-suite-open/group-by.26.query.sqlpp
@@ -20,12 +20,13 @@
 USE TinySocial;
 
 
-SELECT org AS organization,
+SELECT employment.`organization-name` AS organization,
        avg(`get-year`(employment.`end-date`)) AS average,
        count(employment.`end-date`) AS count,
        sum(`get-year`(employment.`end-date`)) AS sum,
        min(employment.`end-date`) AS min,
        max(employment.`end-date`) AS max
 FROM  FacebookUsers as fbu, fbu.employment AS employment
-GROUP BY employment.`organization-name` AS org
+GROUP BY employment.`organization-name`
+ORDER BY employment.`organization-name`
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
index 733924e..86568ca 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.query.sqlpp
@@ -29,4 +29,5 @@ use tpch;
 select element {'l_linenumber':l_linenumber,'count_order':COLL_COUNT(l)}
 from  LineItem as l
 group by l.l_linenumber as l_linenumber
+order by l_linenumber
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
index 0127261..d07b004 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -44,12 +44,13 @@ with  t as (
             from  orders as o
         ))
   )
-select element {'nation_key':nation_key,'sum_price':(
+select element {'nation_key':x.nation_key,'sum_price':(
         select element {'orderdate':y.order_date,'sum_price':y.sum_price}
         from  x as y
         order by y.sum_price desc
         limit 3
     )}
 from  X as x
-group by x.nation_key as nation_key
+group by x.nation_key
+order by x.nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
index db98f32..02edeea 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'nation_key':nation_key,'sum_price':(
+select element {'nation_key':x.nation_key,'sum_price':(
         select element {'orderdate':od,'sum_price':sum}
         from  x as i
         group by i.order_date as od
@@ -48,5 +48,6 @@ from  (
     where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = n.n_nationkey))
     group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
 ) as x
-group by x.nation_key as nation_key
+group by x.nation_key
+order by x.nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue601/query-issue601.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue601/query-issue601.3.query.sqlpp
index fe70e94..b99f103 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue601/query-issue601.3.query.sqlpp
@@ -29,4 +29,5 @@ USE tpch;
 SELECT l_linenumber, count(l) AS count_order
 FROM  LineItem AS l
 GROUP BY l_linenumber
+ORDER BY l_linenumber
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785-2/query-issue785-2.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785-2/query-issue785-2.3.query.sqlpp
index cfe9557..7f17b7a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -50,4 +50,5 @@ SELECT nation_key,
         ) AS sum_price
 FROM  X
 GROUP BY nation_key
+ORDER BY nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785/query-issue785.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785/query-issue785.3.query.sqlpp
index 113df95..c775c9a 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql-sugar/query-issue785/query-issue785.3.query.sqlpp
@@ -44,4 +44,5 @@ FROM  (
         GROUP BY o_orderdate as orderdate, n_nationkey as nation_key
 ) AS x
 GROUP BY nation_key
+ORDER BY nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue601/query-issue601.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue601/query-issue601.3.query.sqlpp
index bd88d13..c3fb2b5 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue601/query-issue601.3.query.sqlpp
@@ -29,4 +29,5 @@ USE tpch;
 SELECT l_linenumber AS l_linenumber, count(l) AS count_order
 FROM  LineItem AS l
 GROUP BY l.l_linenumber AS l_linenumber
+ORDER BY l_linenumber
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785-2/query-issue785-2.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785-2/query-issue785-2.3.query.sqlpp
index 8ac3963..864f97b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -41,7 +41,7 @@ X as (
       GROUP BY orders.o_orderdate AS orderdate, n.n_nationkey AS nation_key
 )
 
-SELECT nation_key As nation_key,
+SELECT x.nation_key As nation_key,
        (
             SELECT y.order_date AS orderdate, y.sum_price As sum_price
             FROM  x AS y
@@ -49,5 +49,6 @@ SELECT nation_key As nation_key,
             LIMIT 3
         ) AS sum_price
 FROM  X AS x
-GROUP BY x.nation_key AS nation_key
+GROUP BY x.nation_key
+ORDER BY x.nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785/query-issue785.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785/query-issue785.3.query.sqlpp
index a142f2a..7cdce6d 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-sql/query-issue785/query-issue785.3.query.sqlpp
@@ -26,7 +26,7 @@
 USE tpch;
 
 
-SELECT  nation_key AS nation_key,
+SELECT  x.nation_key AS nation_key,
         (
             SELECT od AS orderdate, sum_price AS sum_price
             FROM x as i
@@ -43,5 +43,6 @@ FROM  (
         WHERE orders.o_custkey = customer.c_custkey AND customer.c_nationkey = n.n_nationkey
         GROUP BY orders.o_orderdate as orderdate,n.n_nationkey as nation_key
 ) AS x
-GROUP BY x.nation_key AS nation_key
+GROUP BY x.nation_key
+ORDER BY x.nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
index efe6954..4613e75 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,8 @@
 use tpch;
 
 
-select element {'l_linenumber':l_linenumber,'count_order':tpch.count(l)}
+select element {'l_linenumber':l.l_linenumber,'count_order':tpch.count(l)}
 from  LineItem as l
-group by l.l_linenumber as l_linenumber
+group by l.l_linenumber
+order by l.l_linenumber
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
index de434ef..a694e97 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -44,12 +44,13 @@ with  t as (
             from  orders as o
         ))
   )
-select element {'nation_key':nation_key,'sum_price':(
+select element {'nation_key':x.nation_key,'sum_price':(
         select element {'orderdate':y.order_date,'sum_price':y.sum_price}
         from  x as y
         order by y.sum_price desc
         limit 3
     )}
 from  X as x
-group by x.nation_key as nation_key
+group by x.nation_key
+order by x.nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
index c7761f3..214a7c2 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch-with-index/query-issue785/query-issue785.3.query.sqlpp
@@ -26,7 +26,7 @@
 use tpch;
 
 
-select element {'nation_key':nation_key,'sum_price':(
+select element {'nation_key':x.nation_key,'sum_price':(
         select element {'orderdate':od,'sum_price':sum}
         from  x as i
         group by i.order_date as od
@@ -48,5 +48,6 @@ from  (
     where ((orders.o_custkey = customer.c_custkey) and (customer.c_nationkey = n.n_nationkey))
     group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
 ) as x
-group by x.nation_key as nation_key
+group by x.nation_key
+order by x.nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
index efe6954..4613e75 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue601/query-issue601.3.query.sqlpp
@@ -26,7 +26,8 @@
 use tpch;
 
 
-select element {'l_linenumber':l_linenumber,'count_order':tpch.count(l)}
+select element {'l_linenumber':l.l_linenumber,'count_order':tpch.count(l)}
 from  LineItem as l
-group by l.l_linenumber as l_linenumber
+group by l.l_linenumber
+order by l.l_linenumber
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
index de434ef..7141505 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785-2/query-issue785-2.3.query.sqlpp
@@ -52,4 +52,5 @@ select element {'nation_key':nation_key,'sum_price':(
     )}
 from  X as x
 group by x.nation_key as nation_key
+order by nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
index c7761f3..1f5e8e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/tpch/query-issue785/query-issue785.3.query.sqlpp
@@ -49,4 +49,5 @@ from  (
     group by orders.o_orderdate as orderdate,n.n_nationkey as nation_key
 ) as x
 group by x.nation_key as nation_key
+order by nation_key
 ;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/any-object/any-object.2.query.sqlpp
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/any-object/any-object.2.query.sqlpp b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/any-object/any-object.2.query.sqlpp
index 04beba1..91a980e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/any-object/any-object.2.query.sqlpp
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/queries_sqlpp/types/any-object/any-object.2.query.sqlpp
@@ -17,4 +17,6 @@
  * under the License.
  */
 
-select element x from Metadata.Datatype as x;
\ No newline at end of file
+select element x
+from Metadata.Datatype as x
+order by x.DataverseName, x.DatatypeName;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm
index a20b1a7..1ebdc02 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1/cluster_state_1.1.adm
@@ -15,6 +15,7 @@
         "compiler.framesize": 32768,
         "compiler.groupmemory": 163840,
         "compiler.joinmemory": 163840,
+        "compiler.parallelism": 0,
         "compiler.pregelix.home": "~/pregelix",
         "compiler.sortmemory": 327680,
         "core.dump.paths": {},
@@ -125,4 +126,4 @@
     "shutdownUri": "http://127.0.0.1:19002/admin/shutdown",
     "state": "ACTIVE",
     "versionUri": "http://127.0.0.1:19002/admin/version"
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.adm
new file mode 100644
index 0000000..dbdf8f0
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_full/cluster_state_1_full.1.adm
@@ -0,0 +1,129 @@
+{
+    "cc": {
+        "configUri": "http://127.0.0.1:19002/admin/cluster/cc/config",
+        "statsUri": "http://127.0.0.1:19002/admin/cluster/cc/stats",
+        "threadDumpUri": "http://127.0.0.1:19002/admin/cluster/cc/threaddump"
+    },
+    "config": {
+        "api.port": 19002,
+        "cluster.partitions": {
+            "0": "ID:0, Original Node: asterix_nc1, IODevice: 0, Active Node: asterix_nc1",
+            "1": "ID:1, Original Node: asterix_nc1, IODevice: 1, Active Node: asterix_nc1",
+            "2": "ID:2, Original Node: asterix_nc2, IODevice: 0, Active Node: asterix_nc2",
+            "3": "ID:3, Original Node: asterix_nc2, IODevice: 1, Active Node: asterix_nc2"
+        },
+        "compiler.framesize": 32768,
+        "compiler.groupmemory": 163840,
+        "compiler.joinmemory": 163840,
+        "compiler.parallelism": -1,
+        "compiler.pregelix.home": "~/pregelix",
+        "compiler.sortmemory": 327680,
+        "core.dump.paths": {},
+        "feed.central.manager.port": 4500,
+        "feed.max.threshold.period": 5,
+        "feed.memory.available.wait.timeout": 10,
+        "feed.memory.global.budget": 67108864,
+        "feed.pending.work.threshold": 50,
+        "feed.port": 19003,
+        "instance.name": null,
+        "log.level": "INFO",
+        "max.wait.active.cluster": 60,
+        "metadata.callback.port": 0,
+        "metadata.node": "asterix_nc1",
+        "metadata.partition": "ID:0, Original Node: asterix_nc1, IODevice: 0, Active Node: asterix_nc1",
+        "metadata.port": 0,
+        "metadata.registration.timeout.secs": 60,
+        "node.partitions": {
+            "asterix_nc1": [
+                "ID:0, Original Node: asterix_nc1, IODevice: 0, Active Node: asterix_nc1",
+                "ID:1, Original Node: asterix_nc1, IODevice: 1, Active Node: asterix_nc1"
+            ],
+            "asterix_nc2": [
+                "ID:2, Original Node: asterix_nc2, IODevice: 0, Active Node: asterix_nc2",
+                "ID:3, Original Node: asterix_nc2, IODevice: 1, Active Node: asterix_nc2"
+            ]
+        },
+        "node.stores": {
+            "asterix_nc1": [
+                "iodevice0",
+                "iodevice1"
+            ],
+            "asterix_nc2": [
+                "iodevice0",
+                "iodevice1"
+            ]
+        },
+        "plot.activate": false,
+        "storage.buffercache.maxopenfiles": 2147483647,
+        "storage.buffercache.pagesize": 32768,
+        "storage.buffercache.size": 50331648,
+        "storage.lsm.bloomfilter.falsepositiverate": 0.01,
+        "storage.memorycomponent.globalbudget": 536870912,
+        "storage.memorycomponent.numcomponents": 2,
+        "storage.memorycomponent.numpages": 8,
+        "storage.memorycomponent.pagesize": 131072,
+        "storage.metadata.memorycomponent.numpages": 256,
+        "transaction.log.dirs": {
+            "asterix_nc1": "target/txnLogDir/asterix_nc1",
+            "asterix_nc2": "target/txnLogDir/asterix_nc2"
+        },
+        "txn.commitprofiler.reportinterval": 5,
+        "txn.job.recovery.memorysize": 67108864,
+        "txn.lock.escalationthreshold": 1000,
+        "txn.lock.shrinktimer": 5000,
+        "txn.lock.timeout.sweepthreshold": 10000,
+        "txn.lock.timeout.waitthreshold": 60000,
+        "txn.log.buffer.numpages": 8,
+        "txn.log.buffer.pagesize": 131072,
+        "txn.log.checkpoint.history": 0,
+        "txn.log.checkpoint.lsnthreshold": 67108864,
+        "txn.log.checkpoint.pollfrequency": 120,
+        "txn.log.partitionsize": 268435456,
+        "web.port": 19001,
+        "web.queryinterface.port": 19006,
+        "web.secondary.port": 19005
+    },
+    "diagnosticsUri": "http://127.0.0.1:19002/admin/diagnostics",
+    "fullShutdownUri": "http://127.0.0.1:19002/admin/shutdown?all=true",
+    "metadata_node": "asterix_nc1",
+    "ncs": [
+        {
+            "configUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc1/config",
+            "node_id": "asterix_nc1",
+            "partitions": [
+                {
+                    "active": true,
+                    "partition_id": "partition_0"
+                },
+                {
+                    "active": true,
+                    "partition_id": "partition_1"
+                }
+            ],
+            "state": "ACTIVE",
+            "statsUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc1/stats",
+            "threadDumpUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc1/threaddump"
+        },
+        {
+            "configUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc2/config",
+            "node_id": "asterix_nc2",
+            "partitions": [
+                {
+                    "active": true,
+                    "partition_id": "partition_2"
+                },
+                {
+                    "active": true,
+                    "partition_id": "partition_3"
+                }
+            ],
+            "state": "ACTIVE",
+            "statsUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc2/stats",
+            "threadDumpUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc2/threaddump"
+        }
+    ],
+    "replicationUri": "http://127.0.0.1:19002/admin/cluster/replication",
+    "shutdownUri": "http://127.0.0.1:19002/admin/shutdown",
+    "state": "ACTIVE",
+    "versionUri": "http://127.0.0.1:19002/admin/version"
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.adm
new file mode 100644
index 0000000..fd268da
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/api/cluster_state_1_less/cluster_state_1_less.1.adm
@@ -0,0 +1,129 @@
+{
+    "cc": {
+        "configUri": "http://127.0.0.1:19002/admin/cluster/cc/config",
+        "statsUri": "http://127.0.0.1:19002/admin/cluster/cc/stats",
+        "threadDumpUri": "http://127.0.0.1:19002/admin/cluster/cc/threaddump"
+    },
+    "config": {
+        "api.port": 19002,
+        "cluster.partitions": {
+            "0": "ID:0, Original Node: asterix_nc1, IODevice: 0, Active Node: asterix_nc1",
+            "1": "ID:1, Original Node: asterix_nc1, IODevice: 1, Active Node: asterix_nc1",
+            "2": "ID:2, Original Node: asterix_nc2, IODevice: 0, Active Node: asterix_nc2",
+            "3": "ID:3, Original Node: asterix_nc2, IODevice: 1, Active Node: asterix_nc2"
+        },
+        "compiler.framesize": 32768,
+        "compiler.groupmemory": 163840,
+        "compiler.joinmemory": 163840,
+        "compiler.parallelism": 3,
+        "compiler.pregelix.home": "~/pregelix",
+        "compiler.sortmemory": 327680,
+        "core.dump.paths": {},
+        "feed.central.manager.port": 4500,
+        "feed.max.threshold.period": 5,
+        "feed.memory.available.wait.timeout": 10,
+        "feed.memory.global.budget": 67108864,
+        "feed.pending.work.threshold": 50,
+        "feed.port": 19003,
+        "instance.name": null,
+        "log.level": "INFO",
+        "max.wait.active.cluster": 60,
+        "metadata.callback.port": 0,
+        "metadata.node": "asterix_nc1",
+        "metadata.partition": "ID:0, Original Node: asterix_nc1, IODevice: 0, Active Node: asterix_nc1",
+        "metadata.port": 0,
+        "metadata.registration.timeout.secs": 60,
+        "node.partitions": {
+            "asterix_nc1": [
+                "ID:0, Original Node: asterix_nc1, IODevice: 0, Active Node: asterix_nc1",
+                "ID:1, Original Node: asterix_nc1, IODevice: 1, Active Node: asterix_nc1"
+            ],
+            "asterix_nc2": [
+                "ID:2, Original Node: asterix_nc2, IODevice: 0, Active Node: asterix_nc2",
+                "ID:3, Original Node: asterix_nc2, IODevice: 1, Active Node: asterix_nc2"
+            ]
+        },
+        "node.stores": {
+            "asterix_nc1": [
+                "iodevice0",
+                "iodevice1"
+            ],
+            "asterix_nc2": [
+                "iodevice0",
+                "iodevice1"
+            ]
+        },
+        "plot.activate": false,
+        "storage.buffercache.maxopenfiles": 2147483647,
+        "storage.buffercache.pagesize": 32768,
+        "storage.buffercache.size": 50331648,
+        "storage.lsm.bloomfilter.falsepositiverate": 0.01,
+        "storage.memorycomponent.globalbudget": 536870912,
+        "storage.memorycomponent.numcomponents": 2,
+        "storage.memorycomponent.numpages": 8,
+        "storage.memorycomponent.pagesize": 131072,
+        "storage.metadata.memorycomponent.numpages": 256,
+        "transaction.log.dirs": {
+            "asterix_nc1": "target/txnLogDir/asterix_nc1",
+            "asterix_nc2": "target/txnLogDir/asterix_nc2"
+        },
+        "txn.commitprofiler.reportinterval": 5,
+        "txn.job.recovery.memorysize": 67108864,
+        "txn.lock.escalationthreshold": 1000,
+        "txn.lock.shrinktimer": 5000,
+        "txn.lock.timeout.sweepthreshold": 10000,
+        "txn.lock.timeout.waitthreshold": 60000,
+        "txn.log.buffer.numpages": 8,
+        "txn.log.buffer.pagesize": 131072,
+        "txn.log.checkpoint.history": 0,
+        "txn.log.checkpoint.lsnthreshold": 67108864,
+        "txn.log.checkpoint.pollfrequency": 120,
+        "txn.log.partitionsize": 268435456,
+        "web.port": 19001,
+        "web.queryinterface.port": 19006,
+        "web.secondary.port": 19005
+    },
+    "diagnosticsUri": "http://127.0.0.1:19002/admin/diagnostics",
+    "fullShutdownUri": "http://127.0.0.1:19002/admin/shutdown?all=true",
+    "metadata_node": "asterix_nc1",
+    "ncs": [
+        {
+            "configUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc1/config",
+            "node_id": "asterix_nc1",
+            "partitions": [
+                {
+                    "active": true,
+                    "partition_id": "partition_0"
+                },
+                {
+                    "active": true,
+                    "partition_id": "partition_1"
+                }
+            ],
+            "state": "ACTIVE",
+            "statsUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc1/stats",
+            "threadDumpUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc1/threaddump"
+        },
+        {
+            "configUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc2/config",
+            "node_id": "asterix_nc2",
+            "partitions": [
+                {
+                    "active": true,
+                    "partition_id": "partition_2"
+                },
+                {
+                    "active": true,
+                    "partition_id": "partition_3"
+                }
+            ],
+            "state": "ACTIVE",
+            "statsUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc2/stats",
+            "threadDumpUri": "http://127.0.0.1:19002/admin/cluster/node/asterix_nc2/threaddump"
+        }
+    ],
+    "replicationUri": "http://127.0.0.1:19002/admin/cluster/replication",
+    "shutdownUri": "http://127.0.0.1:19002/admin/shutdown",
+    "state": "ACTIVE",
+    "versionUri": "http://127.0.0.1:19002/admin/version"
+}
\ No newline at end of file


[2/8] asterixdb git commit: Supports flexible degree of parallelism.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785/query-issue785.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785/query-issue785.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785/query-issue785.1.adm
index 7324d3c..3ecdc03 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785/query-issue785.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785/query-issue785.1.adm
@@ -1,24 +1,24 @@
-{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
-{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
-{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
-{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
+{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
+{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
+{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
+{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
+{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
+{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
+{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
+{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
 { "nation_key": 13, "sum_price": [ { "orderdate": "1998-02-08", "sum_price": 223537.09 }, { "orderdate": "1993-11-24", "sum_price": 222392.53 }, { "orderdate": "1995-09-13", "sum_price": 197031.52 } ] }
+{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
 { "nation_key": 15, "sum_price": [ { "orderdate": "1998-05-31", "sum_price": 366291.52 }, { "orderdate": "1994-04-24", "sum_price": 228054.01 }, { "orderdate": "1993-01-29", "sum_price": 223995.46 } ] }
 { "nation_key": 16, "sum_price": [ { "orderdate": "1994-09-20", "sum_price": 231012.22 }, { "orderdate": "1992-06-30", "sum_price": 221320.76 }, { "orderdate": "1993-05-14", "sum_price": 207291.83 } ] }
+{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
+{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
-{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
-{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
-{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
-{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
-{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
-{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue601/query-issue601.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue601/query-issue601.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue601/query-issue601.1.adm
index 7d83268..4c3d838 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue601/query-issue601.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue601/query-issue601.1.adm
@@ -1,7 +1,7 @@
-{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 1, "count_order": 1500 }
 { "l_linenumber": 2, "count_order": 1291 }
-{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 3, "count_order": 1077 }
+{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 5, "count_order": 632 }
+{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 7, "count_order": 211 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785-2/query-issue785-2.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785-2/query-issue785-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785-2/query-issue785-2.1.adm
index c6a683d..3d98aaf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785-2/query-issue785-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785-2/query-issue785-2.1.adm
@@ -1,10 +1,10 @@
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22d }, { "orderdate": "1992-07-07", "sum_price": 180692.9d }, { "orderdate": "1996-06-28", "sum_price": 139915.23d } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05d }, { "orderdate": "1995-12-07", "sum_price": 153048.74d }, { "orderdate": "1994-08-22", "sum_price": 147071.86d } ] }
-{ "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31d }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002d }, { "orderdate": "1993-12-24", "sum_price": 211925.95d } ] }
-{ "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85d }, { "orderdate": "1996-08-13", "sum_price": 217709.03d }, { "orderdate": "1992-08-21", "sum_price": 207364.8d } ] }
-{ "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66d }, { "orderdate": "1992-03-04", "sum_price": 219709.6d }, { "orderdate": "1996-01-06", "sum_price": 190490.78d } ] }
-{ "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87d }, { "orderdate": "1997-08-04", "sum_price": 244636.7d }, { "orderdate": "1996-11-20", "sum_price": 222274.54d } ] }
-{ "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69d }, { "orderdate": "1998-07-17", "sum_price": 187156.38d }, { "orderdate": "1993-03-25", "sum_price": 167017.39d } ] }
-{ "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29d }, { "orderdate": "1993-04-11", "sum_price": 221636.83d }, { "orderdate": "1993-05-07", "sum_price": 220715.14d } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88d }, { "orderdate": "1997-01-21", "sum_price": 240284.95d }, { "orderdate": "1997-08-24", "sum_price": 231831.35d } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996d }, { "orderdate": "1995-11-13", "sum_price": 242588.87d }, { "orderdate": "1993-07-15", "sum_price": 214494.39d } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
+{ "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
+{ "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
+{ "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
+{ "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
+{ "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
+{ "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785/query-issue785.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785/query-issue785.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785/query-issue785.1.adm
index 216a8f6..3ecdc03 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785/query-issue785.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-sugar/query-issue785/query-issue785.1.adm
@@ -1,24 +1,24 @@
-{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33d }, { "orderdate": "1997-05-26", "sum_price": 216826.73d }, { "orderdate": "1996-04-30", "sum_price": 180054.29d } ] }
-{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42d }, { "orderdate": "1996-12-24", "sum_price": 237947.61d }, { "orderdate": "1992-12-01", "sum_price": 218116.21d } ] }
-{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06d }, { "orderdate": "1997-02-17", "sum_price": 225518.72d }, { "orderdate": "1996-08-20", "sum_price": 220636.82d } ] }
-{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45d }, { "orderdate": "1992-04-26", "sum_price": 134333.33d }, { "orderdate": "1997-03-09", "sum_price": 132838.49d } ] }
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22d }, { "orderdate": "1992-07-07", "sum_price": 180692.9d }, { "orderdate": "1996-06-28", "sum_price": 139915.23d } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05d }, { "orderdate": "1995-12-07", "sum_price": 153048.74d }, { "orderdate": "1994-08-22", "sum_price": 147071.86d } ] }
-{ "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31d }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002d }, { "orderdate": "1993-12-24", "sum_price": 211925.95d } ] }
-{ "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85d }, { "orderdate": "1996-08-13", "sum_price": 217709.03d }, { "orderdate": "1992-08-21", "sum_price": 207364.8d } ] }
-{ "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66d }, { "orderdate": "1992-03-04", "sum_price": 219709.6d }, { "orderdate": "1996-01-06", "sum_price": 190490.78d } ] }
-{ "nation_key": 13, "sum_price": [ { "orderdate": "1998-02-08", "sum_price": 223537.09d }, { "orderdate": "1993-11-24", "sum_price": 222392.53d }, { "orderdate": "1995-09-13", "sum_price": 197031.52d } ] }
-{ "nation_key": 15, "sum_price": [ { "orderdate": "1998-05-31", "sum_price": 366291.52d }, { "orderdate": "1994-04-24", "sum_price": 228054.01d }, { "orderdate": "1993-01-29", "sum_price": 223995.46d } ] }
-{ "nation_key": 16, "sum_price": [ { "orderdate": "1994-09-20", "sum_price": 231012.22d }, { "orderdate": "1992-06-30", "sum_price": 221320.76d }, { "orderdate": "1993-05-14", "sum_price": 207291.83d } ] }
-{ "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87d }, { "orderdate": "1997-08-04", "sum_price": 244636.7d }, { "orderdate": "1996-11-20", "sum_price": 222274.54d } ] }
-{ "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69d }, { "orderdate": "1998-07-17", "sum_price": 187156.38d }, { "orderdate": "1993-03-25", "sum_price": 167017.39d } ] }
-{ "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29d }, { "orderdate": "1993-04-11", "sum_price": 221636.83d }, { "orderdate": "1993-05-07", "sum_price": 220715.14d } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88d }, { "orderdate": "1997-01-21", "sum_price": 240284.95d }, { "orderdate": "1997-08-24", "sum_price": 231831.35d } ] }
-{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23d }, { "orderdate": "1994-12-03", "sum_price": 234763.73d }, { "orderdate": "1994-09-09", "sum_price": 228002.51d } ] }
-{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56d }, { "orderdate": "1995-03-02", "sum_price": 228136.49d }, { "orderdate": "1992-07-30", "sum_price": 226314.91d } ] }
-{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74d }, { "orderdate": "1997-01-03", "sum_price": 219920.62d }, { "orderdate": "1992-01-02", "sum_price": 210713.88d } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996d }, { "orderdate": "1995-11-13", "sum_price": 242588.87d }, { "orderdate": "1993-07-15", "sum_price": 214494.39d } ] }
-{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02d }, { "orderdate": "1998-07-20", "sum_price": 209155.48d }, { "orderdate": "1994-04-27", "sum_price": 202917.72d } ] }
-{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83d }, { "orderdate": "1992-03-15", "sum_price": 206742.11d }, { "orderdate": "1992-05-10", "sum_price": 203904.8d } ] }
-{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09d }, { "orderdate": "1993-10-31", "sum_price": 224724.11d }, { "orderdate": "1996-04-18", "sum_price": 220727.97d } ] }
-{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74d }, { "orderdate": "1992-06-03", "sum_price": 233161.66d }, { "orderdate": "1996-09-20", "sum_price": 219707.84d } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
+{ "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
+{ "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
+{ "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
+{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
+{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
+{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
+{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
+{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
+{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
+{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
+{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
+{ "nation_key": 13, "sum_price": [ { "orderdate": "1998-02-08", "sum_price": 223537.09 }, { "orderdate": "1993-11-24", "sum_price": 222392.53 }, { "orderdate": "1995-09-13", "sum_price": 197031.52 } ] }
+{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
+{ "nation_key": 15, "sum_price": [ { "orderdate": "1998-05-31", "sum_price": 366291.52 }, { "orderdate": "1994-04-24", "sum_price": 228054.01 }, { "orderdate": "1993-01-29", "sum_price": 223995.46 } ] }
+{ "nation_key": 16, "sum_price": [ { "orderdate": "1994-09-20", "sum_price": 231012.22 }, { "orderdate": "1992-06-30", "sum_price": 221320.76 }, { "orderdate": "1993-05-14", "sum_price": 207291.83 } ] }
+{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
+{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
+{ "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
+{ "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
+{ "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue601/query-issue601.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue601/query-issue601.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue601/query-issue601.1.adm
index 7d83268..4c3d838 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue601/query-issue601.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue601/query-issue601.1.adm
@@ -1,7 +1,7 @@
-{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 1, "count_order": 1500 }
 { "l_linenumber": 2, "count_order": 1291 }
-{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 3, "count_order": 1077 }
+{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 5, "count_order": 632 }
+{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 7, "count_order": 211 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785-2/query-issue785-2.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785-2/query-issue785-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785-2/query-issue785-2.1.adm
index 360adf4..3d98aaf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785-2/query-issue785-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785-2/query-issue785-2.1.adm
@@ -1,10 +1,10 @@
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785/query-issue785.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785/query-issue785.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785/query-issue785.1.adm
index 7324d3c..3ecdc03 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785/query-issue785.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql/query-issue785/query-issue785.1.adm
@@ -1,24 +1,24 @@
-{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
-{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
-{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
-{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
+{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
+{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
+{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
+{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
+{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
+{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
+{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
+{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
 { "nation_key": 13, "sum_price": [ { "orderdate": "1998-02-08", "sum_price": 223537.09 }, { "orderdate": "1993-11-24", "sum_price": 222392.53 }, { "orderdate": "1995-09-13", "sum_price": 197031.52 } ] }
+{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
 { "nation_key": 15, "sum_price": [ { "orderdate": "1998-05-31", "sum_price": 366291.52 }, { "orderdate": "1994-04-24", "sum_price": 228054.01 }, { "orderdate": "1993-01-29", "sum_price": 223995.46 } ] }
 { "nation_key": 16, "sum_price": [ { "orderdate": "1994-09-20", "sum_price": 231012.22 }, { "orderdate": "1992-06-30", "sum_price": 221320.76 }, { "orderdate": "1993-05-14", "sum_price": 207291.83 } ] }
+{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
+{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
-{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
-{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
-{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
-{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
-{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
-{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue601/query-issue601.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue601/query-issue601.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue601/query-issue601.1.adm
index 7d83268..4c3d838 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue601/query-issue601.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue601/query-issue601.1.adm
@@ -1,7 +1,7 @@
-{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 1, "count_order": 1500 }
 { "l_linenumber": 2, "count_order": 1291 }
-{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 3, "count_order": 1077 }
+{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 5, "count_order": 632 }
+{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 7, "count_order": 211 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785-2/query-issue785-2.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785-2/query-issue785-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785-2/query-issue785-2.1.adm
index 360adf4..3d98aaf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785-2/query-issue785-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785-2/query-issue785-2.1.adm
@@ -1,10 +1,10 @@
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785/query-issue785.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785/query-issue785.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785/query-issue785.1.adm
index 7324d3c..3ecdc03 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785/query-issue785.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-with-index/query-issue785/query-issue785.1.adm
@@ -1,24 +1,24 @@
-{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
-{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
-{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
-{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
+{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
+{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
+{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
+{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
+{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
+{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
+{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
+{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
 { "nation_key": 13, "sum_price": [ { "orderdate": "1998-02-08", "sum_price": 223537.09 }, { "orderdate": "1993-11-24", "sum_price": 222392.53 }, { "orderdate": "1995-09-13", "sum_price": 197031.52 } ] }
+{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
 { "nation_key": 15, "sum_price": [ { "orderdate": "1998-05-31", "sum_price": 366291.52 }, { "orderdate": "1994-04-24", "sum_price": 228054.01 }, { "orderdate": "1993-01-29", "sum_price": 223995.46 } ] }
 { "nation_key": 16, "sum_price": [ { "orderdate": "1994-09-20", "sum_price": 231012.22 }, { "orderdate": "1992-06-30", "sum_price": 221320.76 }, { "orderdate": "1993-05-14", "sum_price": 207291.83 } ] }
+{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
+{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
-{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
-{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
-{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
-{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
-{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
-{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue601/query-issue601.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue601/query-issue601.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue601/query-issue601.1.adm
index 7d83268..4c3d838 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue601/query-issue601.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue601/query-issue601.1.adm
@@ -1,7 +1,7 @@
-{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 1, "count_order": 1500 }
 { "l_linenumber": 2, "count_order": 1291 }
-{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 3, "count_order": 1077 }
+{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 5, "count_order": 632 }
+{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 7, "count_order": 211 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785-2/query-issue785-2.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785-2/query-issue785-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785-2/query-issue785-2.1.adm
index 360adf4..3d98aaf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785-2/query-issue785-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785-2/query-issue785-2.1.adm
@@ -1,10 +1,10 @@
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785/query-issue785.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785/query-issue785.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785/query-issue785.1.adm
index 7324d3c..3ecdc03 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785/query-issue785.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch/query-issue785/query-issue785.1.adm
@@ -1,24 +1,24 @@
-{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
-{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
-{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
-{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
+{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
+{ "nation_key": 6, "sum_price": [ { "orderdate": "1992-05-28", "sum_price": 335178.33 }, { "orderdate": "1997-05-26", "sum_price": 216826.73 }, { "orderdate": "1996-04-30", "sum_price": 180054.29 } ] }
+{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
+{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
+{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
+{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
+{ "nation_key": 11, "sum_price": [ { "orderdate": "1994-12-15", "sum_price": 249900.42 }, { "orderdate": "1996-12-24", "sum_price": 237947.61 }, { "orderdate": "1992-12-01", "sum_price": 218116.21 } ] }
+{ "nation_key": 12, "sum_price": [ { "orderdate": "1995-05-01", "sum_price": 245388.06 }, { "orderdate": "1997-02-17", "sum_price": 225518.72 }, { "orderdate": "1996-08-20", "sum_price": 220636.82 } ] }
 { "nation_key": 13, "sum_price": [ { "orderdate": "1998-02-08", "sum_price": 223537.09 }, { "orderdate": "1993-11-24", "sum_price": 222392.53 }, { "orderdate": "1995-09-13", "sum_price": 197031.52 } ] }
+{ "nation_key": 14, "sum_price": [ { "orderdate": "1993-12-27", "sum_price": 230949.45 }, { "orderdate": "1992-04-26", "sum_price": 134333.33 }, { "orderdate": "1997-03-09", "sum_price": 132838.49 } ] }
 { "nation_key": 15, "sum_price": [ { "orderdate": "1998-05-31", "sum_price": 366291.52 }, { "orderdate": "1994-04-24", "sum_price": 228054.01 }, { "orderdate": "1993-01-29", "sum_price": 223995.46 } ] }
 { "nation_key": 16, "sum_price": [ { "orderdate": "1994-09-20", "sum_price": 231012.22 }, { "orderdate": "1992-06-30", "sum_price": 221320.76 }, { "orderdate": "1993-05-14", "sum_price": 207291.83 } ] }
+{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
+{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 8, "sum_price": [ { "orderdate": "1995-07-26", "sum_price": 244704.23 }, { "orderdate": "1994-12-03", "sum_price": 234763.73 }, { "orderdate": "1994-09-09", "sum_price": 228002.51 } ] }
-{ "nation_key": 9, "sum_price": [ { "orderdate": "1992-08-19", "sum_price": 240457.56 }, { "orderdate": "1995-03-02", "sum_price": 228136.49 }, { "orderdate": "1992-07-30", "sum_price": 226314.91 } ] }
-{ "nation_key": 10, "sum_price": [ { "orderdate": "1992-08-15", "sum_price": 232194.74 }, { "orderdate": "1997-01-03", "sum_price": 219920.62 }, { "orderdate": "1992-01-02", "sum_price": 210713.88 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
-{ "nation_key": 5, "sum_price": [ { "orderdate": "1997-04-04", "sum_price": 258779.02 }, { "orderdate": "1998-07-20", "sum_price": 209155.48 }, { "orderdate": "1994-04-27", "sum_price": 202917.72 } ] }
-{ "nation_key": 7, "sum_price": [ { "orderdate": "1995-03-19", "sum_price": 207925.83 }, { "orderdate": "1992-03-15", "sum_price": 206742.11 }, { "orderdate": "1992-05-10", "sum_price": 203904.8 } ] }
-{ "nation_key": 17, "sum_price": [ { "orderdate": "1997-07-05", "sum_price": 233874.09 }, { "orderdate": "1993-10-31", "sum_price": 224724.11 }, { "orderdate": "1996-04-18", "sum_price": 220727.97 } ] }
-{ "nation_key": 18, "sum_price": [ { "orderdate": "1995-10-03", "sum_price": 245976.74 }, { "orderdate": "1992-06-03", "sum_price": 233161.66 }, { "orderdate": "1996-09-20", "sum_price": 219707.84 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.3.ast
index d9d1e6e..cd20331 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.3.ast
@@ -31,47 +31,51 @@ FROM [  (
       (
         LiteralExpr [STRING] [partkey]
         :
-        Variable [ Name=$partkey ]
+        FieldAccessor [
+          Variable [ Name=$i ]
+          Field=l_partkey
+        ]
       )
       (
         LiteralExpr [STRING] [i]
         :
-        Variable [ Name=$i ]
-      )
-    ]
-    ]
-    FROM [      (
-        SELECT ELEMENT [
-        Variable [ Name=$l ]
-        ]
-        FROM [          FunctionCall Metadata.dataset@1[
-            LiteralExpr [STRING] [LineItem]
-          ]
-          AS Variable [ Name=$l ]
-        ]
-        Orderby
+        (
+          SELECT ELEMENT [
           FieldAccessor [
-            Variable [ Name=$l ]
-            Field=l_partkey
+            Variable [ Name=$x ]
+            Field=i
           ]
-          ASC
-          FieldAccessor [
-            Variable [ Name=$l ]
-            Field=l_shipdate
           ]
-          ASC
+          FROM [            Variable [ Name=$g ]
+            AS Variable [ Name=$x ]
+          ]
+          Orderby
+            FieldAccessor [
+              FieldAccessor [
+                Variable [ Name=$x ]
+                Field=i
+              ]
+              Field=l_shipdate
+            ]
+            ASC
 
+        )
       )
+    ]
+    ]
+    FROM [      FunctionCall Metadata.dataset@1[
+        LiteralExpr [STRING] [LineItem]
+      ]
       AS Variable [ Name=$i ]
     ]
     Groupby
-      Variable [ Name=$partkey ]
+      Variable [ Name=$l_partkey ]
       :=
       FieldAccessor [
         Variable [ Name=$i ]
         Field=l_partkey
       ]
-      GROUP AS Variable [ Name=#1 ]
+      GROUP AS Variable [ Name=$g ]
       (
         i:=Variable [ Name=$i ]
       )
@@ -93,3 +97,15 @@ Where
     <
     LiteralExpr [LONG] [4]
   ]
+Orderby
+  FieldAccessor [
+    Variable [ Name=$gen0 ]
+    Field=partkey
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$j ]
+    Field=l_shipdate
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.4.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.4.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.4.ast
index d9d1e6e..cd20331 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.4.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.4.ast
@@ -31,47 +31,51 @@ FROM [  (
       (
         LiteralExpr [STRING] [partkey]
         :
-        Variable [ Name=$partkey ]
+        FieldAccessor [
+          Variable [ Name=$i ]
+          Field=l_partkey
+        ]
       )
       (
         LiteralExpr [STRING] [i]
         :
-        Variable [ Name=$i ]
-      )
-    ]
-    ]
-    FROM [      (
-        SELECT ELEMENT [
-        Variable [ Name=$l ]
-        ]
-        FROM [          FunctionCall Metadata.dataset@1[
-            LiteralExpr [STRING] [LineItem]
-          ]
-          AS Variable [ Name=$l ]
-        ]
-        Orderby
+        (
+          SELECT ELEMENT [
           FieldAccessor [
-            Variable [ Name=$l ]
-            Field=l_partkey
+            Variable [ Name=$x ]
+            Field=i
           ]
-          ASC
-          FieldAccessor [
-            Variable [ Name=$l ]
-            Field=l_shipdate
           ]
-          ASC
+          FROM [            Variable [ Name=$g ]
+            AS Variable [ Name=$x ]
+          ]
+          Orderby
+            FieldAccessor [
+              FieldAccessor [
+                Variable [ Name=$x ]
+                Field=i
+              ]
+              Field=l_shipdate
+            ]
+            ASC
 
+        )
       )
+    ]
+    ]
+    FROM [      FunctionCall Metadata.dataset@1[
+        LiteralExpr [STRING] [LineItem]
+      ]
       AS Variable [ Name=$i ]
     ]
     Groupby
-      Variable [ Name=$partkey ]
+      Variable [ Name=$l_partkey ]
       :=
       FieldAccessor [
         Variable [ Name=$i ]
         Field=l_partkey
       ]
-      GROUP AS Variable [ Name=#1 ]
+      GROUP AS Variable [ Name=$g ]
       (
         i:=Variable [ Name=$i ]
       )
@@ -93,3 +97,15 @@ Where
     <
     LiteralExpr [LONG] [4]
   ]
+Orderby
+  FieldAccessor [
+    Variable [ Name=$gen0 ]
+    Field=partkey
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$j ]
+    Field=l_shipdate
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.5.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.5.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.5.ast
index d9d1e6e..cd20331 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.5.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at00/at00.5.ast
@@ -31,47 +31,51 @@ FROM [  (
       (
         LiteralExpr [STRING] [partkey]
         :
-        Variable [ Name=$partkey ]
+        FieldAccessor [
+          Variable [ Name=$i ]
+          Field=l_partkey
+        ]
       )
       (
         LiteralExpr [STRING] [i]
         :
-        Variable [ Name=$i ]
-      )
-    ]
-    ]
-    FROM [      (
-        SELECT ELEMENT [
-        Variable [ Name=$l ]
-        ]
-        FROM [          FunctionCall Metadata.dataset@1[
-            LiteralExpr [STRING] [LineItem]
-          ]
-          AS Variable [ Name=$l ]
-        ]
-        Orderby
+        (
+          SELECT ELEMENT [
           FieldAccessor [
-            Variable [ Name=$l ]
-            Field=l_partkey
+            Variable [ Name=$x ]
+            Field=i
           ]
-          ASC
-          FieldAccessor [
-            Variable [ Name=$l ]
-            Field=l_shipdate
           ]
-          ASC
+          FROM [            Variable [ Name=$g ]
+            AS Variable [ Name=$x ]
+          ]
+          Orderby
+            FieldAccessor [
+              FieldAccessor [
+                Variable [ Name=$x ]
+                Field=i
+              ]
+              Field=l_shipdate
+            ]
+            ASC
 
+        )
       )
+    ]
+    ]
+    FROM [      FunctionCall Metadata.dataset@1[
+        LiteralExpr [STRING] [LineItem]
+      ]
       AS Variable [ Name=$i ]
     ]
     Groupby
-      Variable [ Name=$partkey ]
+      Variable [ Name=$l_partkey ]
       :=
       FieldAccessor [
         Variable [ Name=$i ]
         Field=l_partkey
       ]
-      GROUP AS Variable [ Name=#1 ]
+      GROUP AS Variable [ Name=$g ]
       (
         i:=Variable [ Name=$i ]
       )
@@ -93,3 +97,15 @@ Where
     <
     LiteralExpr [LONG] [4]
   ]
+Orderby
+  FieldAccessor [
+    Variable [ Name=$gen0 ]
+    Field=partkey
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$j ]
+    Field=l_shipdate
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast
index dfeebe0..d913c8b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/flwor/at06/at06.3.ast
@@ -39,42 +39,41 @@ FROM [  (
       (
         LiteralExpr [STRING] [i]
         :
-        Variable [ Name=$i ]
+        (
+          SELECT ELEMENT [
+          FieldAccessor [
+            Variable [ Name=$x ]
+            Field=i
+          ]
+          ]
+          FROM [            Variable [ Name=$g ]
+            AS Variable [ Name=$x ]
+          ]
+        )
       )
       (
         LiteralExpr [STRING] [partkey]
         :
-        Variable [ Name=$partkey ]
+        FieldAccessor [
+          Variable [ Name=$i ]
+          Field=l_partkey
+        ]
       )
     ]
     ]
-    FROM [      (
-        SELECT ELEMENT [
-        Variable [ Name=$l ]
-        ]
-        FROM [          FunctionCall Metadata.dataset@1[
-            LiteralExpr [STRING] [LineItem]
-          ]
-          AS Variable [ Name=$l ]
-        ]
-        Orderby
-          FieldAccessor [
-            Variable [ Name=$l ]
-            Field=l_partkey
-          ]
-          ASC
-
-      )
+    FROM [      FunctionCall Metadata.dataset@1[
+        LiteralExpr [STRING] [LineItem]
+      ]
       AS Variable [ Name=$i ]
     ]
     Groupby
-      Variable [ Name=$partkey ]
+      Variable [ Name=$l_partkey ]
       :=
       FieldAccessor [
         Variable [ Name=$i ]
         Field=l_partkey
       ]
-      GROUP AS Variable [ Name=#1 ]
+      GROUP AS Variable [ Name=$g ]
       (
         i:=Variable [ Name=$i ]
       )
@@ -98,6 +97,11 @@ FROM [  (
         Field=l_shipdate
       ]
       ASC
+      FieldAccessor [
+        Variable [ Name=$ii ]
+        Field=l_orderkey
+      ]
+      ASC
 
   )
   AS Variable [ Name=$j ]
@@ -110,3 +114,20 @@ Where
     <
     LiteralExpr [LONG] [4]
   ]
+Orderby
+  FieldAccessor [
+    Variable [ Name=$gen0 ]
+    Field=partkey
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$j ]
+    Field=l_shipdate
+  ]
+  ASC
+  FieldAccessor [
+    Variable [ Name=$j ]
+    Field=l_orderkey
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue456/query-issue456.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue456/query-issue456.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue456/query-issue456.3.ast
index 1dc9f40..1113fa8 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue456/query-issue456.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/open-closed/query-issue456/query-issue456.3.ast
@@ -2,7 +2,10 @@ DataverseUse test
 Query:
 SELECT ELEMENT [
 OrderedListConstructor [
-  Variable [ Name=$m ]
+  FieldAccessor [
+    Variable [ Name=$x ]
+    Field=int_m
+  ]
   FunctionCall test.sql-count@1[
     (
       SELECT ELEMENT [
@@ -29,7 +32,7 @@ Let Variable [ Name=$id ]
     Field=id
   ]
 Groupby
-  Variable [ Name=$m ]
+  Variable [ Name=$int_m ]
   :=
   FieldAccessor [
     Variable [ Name=$x ]
@@ -40,3 +43,10 @@ Groupby
     x:=Variable [ Name=$x ]
   )
 
+Orderby
+  FieldAccessor [
+    Variable [ Name=$x ]
+    Field=int_m
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.26.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.26.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.26.ast
index 16e5ee8..f94d3dc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.26.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tinysocial/tinysocial-suite-open/tinysocial-suite.26.ast
@@ -1,7 +1,10 @@
 DataverseUse TinySocial
 Query:
 SELECT [
-Variable [ Name=$org ]
+FieldAccessor [
+  Variable [ Name=$employment ]
+  Field=organization-name
+]
 organization
 FunctionCall TinySocial.sql-avg@1[
   (
@@ -105,7 +108,7 @@ FROM [  FunctionCall Metadata.dataset@1[
   AS Variable [ Name=$employment ]
 ]
 Groupby
-  Variable [ Name=$org ]
+  Variable [ Name=$organization-name ]
   :=
   FieldAccessor [
     Variable [ Name=$employment ]
@@ -117,3 +120,10 @@ Groupby
     employment:=Variable [ Name=$employment ]
   )
 
+Orderby
+  FieldAccessor [
+    Variable [ Name=$employment ]
+    Field=organization-name
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.ast
index 758fd5a..6da2337 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue601/query-issue601.3.ast
@@ -33,3 +33,7 @@ Groupby
     l:=Variable [ Name=$l ]
   )
 
+Orderby
+  Variable [ Name=$l_linenumber ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.ast
index 991c38a..b56a7bc 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785-2/query-issue785-2.3.ast
@@ -148,7 +148,10 @@ RecordConstructor [
   (
     LiteralExpr [STRING] [nation_key]
     :
-    Variable [ Name=$nation_key ]
+    FieldAccessor [
+      Variable [ Name=$x ]
+      Field=nation_key
+    ]
   )
   (
     LiteralExpr [STRING] [sum_price]
@@ -205,3 +208,10 @@ Groupby
     x:=Variable [ Name=$x ]
   )
 
+Orderby
+  FieldAccessor [
+    Variable [ Name=$x ]
+    Field=nation_key
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.ast
index 61e0ca4..59669e3 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch-sql-like/query-issue785/query-issue785.3.ast
@@ -5,7 +5,10 @@ RecordConstructor [
   (
     LiteralExpr [STRING] [nation_key]
     :
-    Variable [ Name=$nation_key ]
+    FieldAccessor [
+      Variable [ Name=$x ]
+      Field=nation_key
+    ]
   )
   (
     LiteralExpr [STRING] [sum_price]
@@ -173,3 +176,10 @@ Groupby
     x:=Variable [ Name=$x ]
   )
 
+Orderby
+  FieldAccessor [
+    Variable [ Name=$x ]
+    Field=nation_key
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue601/query-issue601.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue601/query-issue601.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue601/query-issue601.3.ast
index ed17476..35785dd 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue601/query-issue601.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue601/query-issue601.3.ast
@@ -5,7 +5,10 @@ RecordConstructor [
   (
     LiteralExpr [STRING] [l_linenumber]
     :
-    Variable [ Name=$l_linenumber ]
+    FieldAccessor [
+      Variable [ Name=$l ]
+      Field=l_linenumber
+    ]
   )
   (
     LiteralExpr [STRING] [count_order]
@@ -43,3 +46,10 @@ Groupby
     l:=Variable [ Name=$l ]
   )
 
+Orderby
+  FieldAccessor [
+    Variable [ Name=$l ]
+    Field=l_linenumber
+  ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785-2/query-issue785-2.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785-2/query-issue785-2.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785-2/query-issue785-2.3.ast
index 991c38a..4ff201e 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785-2/query-issue785-2.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785-2/query-issue785-2.3.ast
@@ -205,3 +205,7 @@ Groupby
     x:=Variable [ Name=$x ]
   )
 
+Orderby
+  Variable [ Name=$nation_key ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785/query-issue785.3.ast
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785/query-issue785.3.ast b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785/query-issue785.3.ast
index 61e0ca4..066626b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785/query-issue785.3.ast
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results_parser_sqlpp/tpch/query-issue785/query-issue785.3.ast
@@ -173,3 +173,7 @@ Groupby
     x:=Variable [ Name=$x ]
   )
 
+Orderby
+  Variable [ Name=$nation_key ]
+  ASC
+

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/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 14cf70b..4e18bde 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
@@ -18,12 +18,13 @@
  */
 package org.apache.asterix.common.config;
 
-import org.apache.hyracks.util.StorageUtil;
-
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.KILOBYTE;
 import static org.apache.hyracks.util.StorageUtil.StorageUnit.MEGABYTE;
 
+import org.apache.hyracks.util.StorageUtil;
+
 public class CompilerProperties extends AbstractProperties {
+
     private static final String COMPILER_SORTMEMORY_KEY = "compiler.sortmemory";
     private static final long COMPILER_SORTMEMORY_DEFAULT = StorageUtil.getSizeInBytes(32, MEGABYTE);
 
@@ -36,6 +37,9 @@ public class CompilerProperties extends AbstractProperties {
     private static final String COMPILER_FRAMESIZE_KEY = "compiler.framesize";
     private static final int COMPILER_FRAMESIZE_DEFAULT = StorageUtil.getSizeInBytes(32, KILOBYTE);
 
+    private static final String COMPILER_PARALLELISM_KEY = "compiler.parallelism";
+    public static final int COMPILER_PARALLELISM_AS_STORAGE = 0;
+
     private static final String COMPILER_PREGELIX_HOME = "compiler.pregelix.home";
     private static final String COMPILER_PREGELIX_HOME_DEFAULT = "~/pregelix";
 
@@ -67,6 +71,12 @@ public class CompilerProperties extends AbstractProperties {
                 PropertyInterpreters.getIntegerBytePropertyInterpreter());
     }
 
+    @PropertyKey(COMPILER_PARALLELISM_KEY)
+    public int getParallelism() {
+        return accessor.getProperty(COMPILER_PARALLELISM_KEY, COMPILER_PARALLELISM_AS_STORAGE,
+                PropertyInterpreters.getIntegerPropertyInterpreter());
+    }
+
     @PropertyKey(COMPILER_PREGELIX_HOME)
     public String getPregelixHome() {
         return accessor.getProperty(COMPILER_PREGELIX_HOME, COMPILER_PREGELIX_HOME_DEFAULT,


[8/8] asterixdb git commit: Supports flexible degree of parallelism.

Posted by bu...@apache.org.
Supports flexible degree of parallelism.

Detailed list of changes include:
- Add the IClusterInfoCollector interface and feed that to the compiler;
- Add the number of cores into the heartbeat;
- Add partition calculation in APIFramework;
- Fix partitioning property requirements in binary operators such as Join and UnionAll;
- Fix PushSubplanIntoGroupByRule for type propagation;
- Fix InlineLeftNtsInSubplanJoinFlatteningVisitor for the returned top join operator;
- Add unit test for partition calculation in APIFrameworkTest
- Add integration test SqlppExecutionFullParallelismIT, SqlppExecutionLessParallelismIT,
  AqlExecutionFullParallelismIT, and AqlExecutionLessParallelismIT for different
  parallelism settings;
- Refactor AQLExecutionTest and SqlppExecutionTest;
- Separate one cluster state test out from AqlExecutionTest.

Change-Id: If70faf52bed995c7098cc343f2fabd1b6c8d96e7
Reviewed-on: https://asterix-gerrit.ics.uci.edu/1398
Tested-by: Jenkins <je...@fulliautomatix.ics.uci.edu>
BAD: Jenkins <je...@fulliautomatix.ics.uci.edu>
Integration-Tests: Jenkins <je...@fulliautomatix.ics.uci.edu>
Reviewed-by: Till Westmann <ti...@apache.org>


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

Branch: refs/heads/master
Commit: 03e940a7781dbdbb654fe13ec7077aae74e1c416
Parents: 35ea53b
Author: Yingyi Bu <yi...@couchbase.com>
Authored: Wed Dec 21 14:18:52 2016 -0800
Committer: Yingyi Bu <bu...@gmail.com>
Committed: Wed Dec 21 17:03:51 2016 -0800

----------------------------------------------------------------------
 ...neLeftNtsInSubplanJoinFlatteningVisitor.java |  12 +-
 .../asterix/translator/IStatementExecutor.java  |   8 +-
 .../apache/asterix/api/common/APIFramework.java |  66 +-
 .../asterix/app/translator/QueryTranslator.java |  22 +-
 .../resources/asterix-build-configuration.xml   |   4 -
 .../resources/asterix-build-configuration2.xml  | 116 +++
 .../resources/asterix-build-configuration3.xml  | 116 +++
 .../asterix/api/common/APIFrameworkTest.java    | 134 +++
 .../http/servlet/ConnectorAPIServletTest.java   |   6 +-
 .../api/http/servlet/VersionAPIServletTest.java |  18 +-
 .../runtime/AqlExecutionFullParallelismIT.java  |  65 ++
 .../runtime/AqlExecutionLessParallelismIT.java  |  65 ++
 .../asterix/test/runtime/AqlExecutionTest.java  |  65 ++
 ...lusterStateExecutionFullParallelismTest.java |  65 ++
 ...lusterStateExecutionLessParallelismTest.java |  65 ++
 .../test/runtime/ClusterStateExecutionTest.java |  65 ++
 .../asterix/test/runtime/ExecutionTest.java     | 139 ---
 .../asterix/test/runtime/ExecutionTestUtil.java |   2 +-
 .../asterix/test/runtime/LangExecutionUtil.java | 115 +++
 .../asterix/test/runtime/RepeatedTest.java      |   9 +-
 .../SqlppExecutionFullParallelismIT.java        |  63 ++
 .../SqlppExecutionLessParallelismIT.java        |  65 ++
 .../test/runtime/SqlppExecutionTest.java        |  56 +-
 .../results/nonpure/query-ASTERIXDB-1608.plan   |  16 +-
 .../optimizerts/results/query-issue827-2.plan   |  44 +-
 .../test/resources/runtimets/cluster_state.xml  |  28 +
 .../resources/runtimets/cluster_state_full.xml  |  28 +
 .../resources/runtimets/cluster_state_less.xml  |  28 +
 .../resources/runtimets/only_cluster_state.xml  |  25 +
 .../runtimets/queries/api/APIQueries.xml        |   5 -
 .../queries/flwor/at00/at00.3.query.aql         |   6 +-
 .../queries/flwor/at00/at00.4.asyncdefer.aql    |   4 +-
 .../queries/flwor/at00/at00.5.async.aql         |   4 +-
 .../queries/flwor/at06/at06.3.query.aql         |   4 +-
 .../query-issue456/query-issue456.3.query.aql   |   3 +-
 .../query-issue601/query-issue601.3.query.aql   |   1 +
 .../query-issue785-2.3.query.aql                |   1 +
 .../query-issue785/query-issue785.3.query.aql   |   1 +
 .../query-issue601/query-issue601.3.query.aql   |   1 +
 .../query-issue785-2.3.query.aql                |   3 +-
 .../query-issue785/query-issue785.3.query.aql   |   1 +
 .../queries_sqlpp/flwor/at00/at00.3.query.sqlpp |   9 +-
 .../flwor/at00/at00.4.asyncdefer.sqlpp          |   9 +-
 .../queries_sqlpp/flwor/at00/at00.5.async.sqlpp |  10 +-
 .../queries_sqlpp/flwor/at06/at06.3.query.sqlpp |  15 +-
 .../query-issue456/query-issue456.3.query.sqlpp |   5 +-
 .../group-by.26.query.sqlpp                     |   5 +-
 .../query-issue601/query-issue601.3.query.sqlpp |   1 +
 .../query-issue785-2.3.query.sqlpp              |   5 +-
 .../query-issue785/query-issue785.3.query.sqlpp |   5 +-
 .../query-issue601/query-issue601.3.query.sqlpp |   1 +
 .../query-issue785-2.3.query.sqlpp              |   1 +
 .../query-issue785/query-issue785.3.query.sqlpp |   1 +
 .../query-issue601/query-issue601.3.query.sqlpp |   1 +
 .../query-issue785-2.3.query.sqlpp              |   5 +-
 .../query-issue785/query-issue785.3.query.sqlpp |   5 +-
 .../query-issue601/query-issue601.3.query.sqlpp |   5 +-
 .../query-issue785-2.3.query.sqlpp              |   5 +-
 .../query-issue785/query-issue785.3.query.sqlpp |   5 +-
 .../query-issue601/query-issue601.3.query.sqlpp |   5 +-
 .../query-issue785-2.3.query.sqlpp              |   1 +
 .../query-issue785/query-issue785.3.query.sqlpp |   1 +
 .../types/any-object/any-object.2.query.sqlpp   |   4 +-
 .../api/cluster_state_1/cluster_state_1.1.adm   |   3 +-
 .../cluster_state_1_full.1.adm                  | 129 +++
 .../cluster_state_1_less.1.adm                  | 129 +++
 .../runtimets/results/flwor/at00/at00.1.adm     | 920 +++++++++---------
 .../runtimets/results/flwor/at00/at00.2.adm     | 920 +++++++++---------
 .../runtimets/results/flwor/at00/at00.3.adm     | 920 +++++++++---------
 .../runtimets/results/flwor/at06/at06.1.adm     | 922 +++++++++----------
 .../tinysocial-suite-open/group-by.26.adm       |  12 +-
 .../query-issue601/query-issue601.1.adm         |   4 +-
 .../query-issue785-2/query-issue785-2.1.adm     |   8 +-
 .../query-issue785/query-issue785.1.adm         |  30 +-
 .../query-issue601/query-issue601.1.adm         |   4 +-
 .../query-issue785-2/query-issue785-2.1.adm     |  20 +-
 .../query-issue785/query-issue785.1.adm         |  48 +-
 .../query-issue601/query-issue601.1.adm         |   4 +-
 .../query-issue785-2/query-issue785-2.1.adm     |   8 +-
 .../query-issue785/query-issue785.1.adm         |  30 +-
 .../query-issue601/query-issue601.1.adm         |   4 +-
 .../query-issue785-2/query-issue785-2.1.adm     |   8 +-
 .../query-issue785/query-issue785.1.adm         |  30 +-
 .../tpch/query-issue601/query-issue601.1.adm    |   4 +-
 .../query-issue785-2/query-issue785-2.1.adm     |   8 +-
 .../tpch/query-issue785/query-issue785.1.adm    |  30 +-
 .../results_parser_sqlpp/flwor/at00/at00.3.ast  |  64 +-
 .../results_parser_sqlpp/flwor/at00/at00.4.ast  |  64 +-
 .../results_parser_sqlpp/flwor/at00/at00.5.ast  |  64 +-
 .../results_parser_sqlpp/flwor/at06/at06.3.ast  |  63 +-
 .../query-issue456/query-issue456.3.ast         |  14 +-
 .../tinysocial-suite.26.ast                     |  14 +-
 .../query-issue601/query-issue601.3.ast         |   4 +
 .../query-issue785-2/query-issue785-2.3.ast     |  12 +-
 .../query-issue785/query-issue785.3.ast         |  12 +-
 .../tpch/query-issue601/query-issue601.3.ast    |  12 +-
 .../query-issue785-2/query-issue785-2.3.ast     |   4 +
 .../tpch/query-issue785/query-issue785.3.ast    |   4 +
 .../common/config/CompilerProperties.java       |  14 +-
 .../physical/AbstractHashJoinPOperator.java     |  49 +-
 .../physical/NestedLoopJoinPOperator.java       |   9 +-
 .../operators/physical/UnionAllPOperator.java   |  34 +-
 .../properties/StructuralPropertiesVector.java  |   3 +-
 .../algebra/util/OperatorPropertiesUtil.java    |  20 +
 .../subplan/PushSubplanIntoGroupByRule.java     |   3 +-
 .../hyracks/api/client/HyracksConnection.java   |  16 +-
 .../api/client/IClusterInfoCollector.java       |  46 +
 .../api/client/IHyracksClientConnection.java    |  30 +-
 .../hyracks/api/client/NodeControllerInfo.java  |   9 +-
 .../hyracks/control/cc/NodeControllerState.java |   8 +
 .../cc/work/GetNodeControllersInfoWork.java     |   5 +-
 .../control/common/heartbeat/HeartbeatData.java |   3 +
 .../control/nc/NodeControllerService.java       |   1 +
 .../std/union/UnionAllOperatorDescriptor.java   |  12 +-
 .../hyracks/hdfs/scheduler/SchedulerTest.java   |   4 +-
 .../apache/hyracks/hdfs/utils/TestUtils.java    |   2 +-
 116 files changed, 3951 insertions(+), 2491 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/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 d3a0c0f..6a318d6 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
@@ -38,10 +38,10 @@ import org.apache.hyracks.algebricks.core.algebra.operators.logical.AbstractBina
 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.GroupByOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.InnerJoinOperator;
 import org.apache.hyracks.algebricks.core.algebra.operators.logical.IntersectOperator;
@@ -52,7 +52,6 @@ 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;
@@ -64,6 +63,7 @@ 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;
 
@@ -381,11 +381,9 @@ class InlineLeftNtsInSubplanJoinFlatteningVisitor implements IQueryOperatorVisit
             // Deals with single input operators.
             Mutable<ILogicalOperator> childRef = op.getInputs().get(0);
             ILogicalOperator newChild = childRef.getValue().accept(this, null);
-            if (topJoinRef == null) {
-                LogicalOperatorTag childOpTag = newChild.getOperatorTag();
-                if (childOpTag == LogicalOperatorTag.INNERJOIN || childOpTag == LogicalOperatorTag.LEFTOUTERJOIN) {
-                    topJoinRef = childRef;
-                }
+            LogicalOperatorTag childOpTag = newChild.getOperatorTag();
+            if (childOpTag == LogicalOperatorTag.INNERJOIN || childOpTag == LogicalOperatorTag.LEFTOUTERJOIN) {
+                topJoinRef = childRef;
             }
             op.getInputs().get(0).setValue(newChild);
         }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
index a1f3055..2066f73 100644
--- a/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
+++ b/asterixdb/asterix-algebra/src/main/java/org/apache/asterix/translator/IStatementExecutor.java
@@ -26,6 +26,7 @@ import org.apache.asterix.lang.common.statement.Query;
 import org.apache.asterix.metadata.declared.MetadataProvider;
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
+import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.dataset.IHyracksDataset;
 import org.apache.hyracks.api.job.JobSpecification;
@@ -108,6 +109,8 @@ public interface IStatementExecutor {
     /**
      * rewrites and compiles query into a hyracks job specifications
      *
+     * @param clusterInfoCollector
+     *            The cluster info collector
      * @param metadataProvider
      *            The metadataProvider used to access metadata and build runtimes
      * @param query
@@ -120,9 +123,8 @@ public interface IStatementExecutor {
      * @throws AlgebricksException
      * @throws ACIDException
      */
-    JobSpecification rewriteCompileQuery(MetadataProvider metadataProvider, Query query,
-            ICompiledDmlStatement dmlStatement)
-                    throws AsterixException, RemoteException, AlgebricksException, ACIDException;
+    JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector, MetadataProvider metadataProvider,
+            Query query, ICompiledDmlStatement dmlStatement) throws RemoteException, AlgebricksException, ACIDException;
 
     /**
      * returns the active dataverse for an entity or a statement

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/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 e27edb1..16646d2 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
@@ -18,10 +18,18 @@
  */
 package org.apache.asterix.api.common;
 
+
 import java.io.IOException;
 import java.io.PrintWriter;
+import java.rmi.Remote;
 import java.rmi.RemoteException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.Set;
 
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslator;
 import org.apache.asterix.algebra.base.ILangExpressionToPlanTranslatorFactory;
@@ -55,6 +63,7 @@ import org.apache.asterix.transaction.management.service.transaction.JobIdFactor
 import org.apache.asterix.translator.CompiledStatements.ICompiledDmlStatement;
 import org.apache.asterix.translator.IStatementExecutor.Stats;
 import org.apache.asterix.translator.SessionConfig;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
 import org.apache.hyracks.algebricks.common.constraints.AlgebricksPartitionConstraint;
 import org.apache.hyracks.algebricks.common.exceptions.AlgebricksException;
 import org.apache.hyracks.algebricks.common.utils.Pair;
@@ -76,7 +85,9 @@ import org.apache.hyracks.algebricks.core.algebra.prettyprint.PlanPrettyPrinter;
 import org.apache.hyracks.algebricks.core.rewriter.base.AlgebricksOptimizationContext;
 import org.apache.hyracks.algebricks.core.rewriter.base.IOptimizationContextFactory;
 import org.apache.hyracks.algebricks.core.rewriter.base.PhysicalOptimizationConfig;
+import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
+import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.job.JobId;
 import org.apache.hyracks.api.job.JobSpecification;
 import org.json.JSONException;
@@ -150,7 +161,7 @@ public class APIFramework {
         return new Pair<>(q, q.getVarCounter());
     }
 
-    public JobSpecification compileQuery(List<FunctionDecl> declaredFunctions,
+    public JobSpecification compileQuery(IClusterInfoCollector clusterInfoCollector,
             MetadataProvider metadataProvider, Query rwQ, int varCounter, String outputDatasetName,
             SessionConfig conf, ICompiledDmlStatement statement)
             throws AlgebricksException, RemoteException, ACIDException {
@@ -219,7 +230,10 @@ public class APIFramework {
         builder.setExpressionTypeComputer(ExpressionTypeComputer.INSTANCE);
         builder.setMissableTypeComputer(AqlMissableTypeComputer.INSTANCE);
         builder.setConflictingTypeResolver(ConflictingTypeResolver.INSTANCE);
-        builder.setClusterLocations(metadataProvider.getClusterLocations());
+
+        int parallelism = compilerProperties.getParallelism();
+        builder.setClusterLocations(parallelism == CompilerProperties.COMPILER_PARALLELISM_AS_STORAGE
+                ? metadataProvider.getClusterLocations() : getComputationLocations(clusterInfoCollector, parallelism));
 
         ICompiler compiler = compilerFactory.createCompiler(plan, metadataProvider, t.getVarCounter());
         if (conf.isOptimize()) {
@@ -342,4 +356,52 @@ public class APIFramework {
             out.println("<pre>Duration: " + duration + " sec</pre>");
         }
     }
+
+    // Computes the location constraints based on user-configured parallelism parameter.
+    // Note that the parallelism parameter is only a hint -- it will not be respected if it is too small or too large.
+    private AlgebricksAbsolutePartitionConstraint getComputationLocations(IClusterInfoCollector clusterInfoCollector,
+            int parallelismHint) throws AlgebricksException {
+        try {
+            Map<String, NodeControllerInfo> ncMap = clusterInfoCollector.getNodeControllerInfos();
+
+            // Unifies the handling of non-positive parallelism.
+            int parallelism = parallelismHint <= 0 ? -2 * ncMap.size() : parallelismHint;
+
+            // Calculates per node parallelism, with load balance, i.e., randomly selecting nodes with larger
+            // parallelism.
+            int numNodes = ncMap.size();
+            int numNodesWithOneMorePartition = parallelism % numNodes;
+            int perNodeParallelismMin = parallelism / numNodes;
+            int perNodeParallelismMax = parallelism / numNodes + 1;
+            List<String> allNodes = new ArrayList<>();
+            Set<String> selectedNodesWithOneMorePartition = new HashSet<>();
+            for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
+                allNodes.add(entry.getKey());
+            }
+            Random random = new Random();
+            for (int index = numNodesWithOneMorePartition; index >= 1; --index) {
+                int pick = random.nextInt(index);
+                selectedNodesWithOneMorePartition.add(allNodes.get(pick));
+                Collections.swap(allNodes, pick, index - 1);
+            }
+
+            // Generates cluster locations, which has duplicates for a node if it contains more than one partitions.
+            List<String> locations = new ArrayList<>();
+            for (Map.Entry<String, NodeControllerInfo> entry : ncMap.entrySet()) {
+                String nodeId = entry.getKey();
+                int numCores = entry.getValue().getNumCores();
+                int availableCores = numCores > 1 ? numCores - 1 : numCores; // Reserves one core for heartbeat.
+                int nodeParallelism = selectedNodesWithOneMorePartition.contains(nodeId) ? perNodeParallelismMax
+                        : perNodeParallelismMin;
+                int coresToUse = nodeParallelism >= 0 && nodeParallelism < availableCores ? nodeParallelism
+                        : availableCores;
+                for (int count = 0; count < coresToUse; ++count) {
+                    locations.add(nodeId);
+                }
+            }
+            return new AlgebricksAbsolutePartitionConstraint(locations.toArray(new String[0]));
+        } catch (Exception e) {
+            throw new AlgebricksException(e);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
index 643a352..4fab8d7 100644
--- a/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
+++ b/asterixdb/asterix-app/src/main/java/org/apache/asterix/app/translator/QueryTranslator.java
@@ -53,13 +53,13 @@ import org.apache.asterix.app.external.FeedOperations;
 import org.apache.asterix.app.result.ResultHandle;
 import org.apache.asterix.app.result.ResultReader;
 import org.apache.asterix.app.result.ResultUtil;
-import org.apache.asterix.common.config.ExternalProperties;
 import org.apache.asterix.common.config.ClusterProperties;
+import org.apache.asterix.common.config.ExternalProperties;
+import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.config.DatasetConfig.DatasetType;
 import org.apache.asterix.common.config.DatasetConfig.ExternalDatasetTransactionState;
 import org.apache.asterix.common.config.DatasetConfig.ExternalFilePendingOp;
 import org.apache.asterix.common.config.DatasetConfig.IndexType;
-import org.apache.asterix.common.config.GlobalConfig;
 import org.apache.asterix.common.exceptions.ACIDException;
 import org.apache.asterix.common.exceptions.AsterixException;
 import org.apache.asterix.common.functions.FunctionSignature;
@@ -68,8 +68,8 @@ import org.apache.asterix.external.api.IAdapterFactory;
 import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber;
 import org.apache.asterix.external.feed.api.IActiveLifecycleEventSubscriber.ActiveLifecycleEvent;
 import org.apache.asterix.external.feed.api.IFeed;
-import org.apache.asterix.external.feed.api.IFeed.FeedType;
 import org.apache.asterix.external.feed.api.IFeedJoint;
+import org.apache.asterix.external.feed.api.IFeed.FeedType;
 import org.apache.asterix.external.feed.api.IFeedJoint.FeedJointType;
 import org.apache.asterix.external.feed.management.ActiveLifecycleEventSubscriber;
 import org.apache.asterix.external.feed.management.FeedConnectionId;
@@ -193,6 +193,7 @@ import org.apache.hyracks.algebricks.data.IAWriterFactory;
 import org.apache.hyracks.algebricks.data.IResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.runtime.serializer.ResultSerializerFactoryProvider;
 import org.apache.hyracks.algebricks.runtime.writers.PrinterBasedWriterFactory;
+import org.apache.hyracks.api.client.IClusterInfoCollector;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.dataflow.value.ITypeTraits;
 import org.apache.hyracks.api.dataset.IHyracksDataset;
@@ -1836,7 +1837,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             CompiledLoadFromFileStatement cls = new CompiledLoadFromFileStatement(dataverseName,
                     loadStmt.getDatasetName().getValue(), loadStmt.getAdapter(), loadStmt.getProperties(),
                     loadStmt.dataIsAlreadySorted());
-            JobSpecification spec = apiFramework.compileQuery(null, metadataProvider, null, 0, null, sessionConfig,
+            JobSpecification spec = apiFramework.compileQuery(hcc, metadataProvider, null, 0, null, sessionConfig,
                     cls);
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1896,7 +1897,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 default:
                     throw new AlgebricksException("Unsupported statement type " + stmtInsertUpsert.getKind());
             }
-            JobSpecification jobSpec = rewriteCompileQuery(metadataProvider, query, clfrqs);
+            JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, clfrqs);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1938,7 +1939,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
             CompiledDeleteStatement clfrqs = new CompiledDeleteStatement(stmtDelete.getVariableExpr(), dataverseName,
                     stmtDelete.getDatasetName().getValue(), stmtDelete.getCondition(), stmtDelete.getVarCounter(),
                     stmtDelete.getQuery());
-            JobSpecification jobSpec = rewriteCompileQuery(metadataProvider, clfrqs.getQuery(), clfrqs);
+            JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, clfrqs.getQuery(), clfrqs);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;
@@ -1960,7 +1961,8 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
     }
 
     @Override
-    public JobSpecification rewriteCompileQuery(MetadataProvider metadataProvider, Query query,
+    public JobSpecification rewriteCompileQuery(IClusterInfoCollector clusterInfoCollector,
+            MetadataProvider metadataProvider, Query query,
             ICompiledDmlStatement stmt)
             throws AsterixException, RemoteException, AlgebricksException, ACIDException {
 
@@ -1969,7 +1971,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
                 sessionConfig);
 
         // Query Compilation (happens under the same ongoing metadata transaction)
-        JobSpecification spec = apiFramework.compileQuery(declaredFunctions, metadataProvider, reWrittenQuery.first,
+        JobSpecification spec = apiFramework.compileQuery(clusterInfoCollector, metadataProvider, reWrittenQuery.first,
                 reWrittenQuery.second, stmt == null ? null : stmt.getDatasetName(), sessionConfig, stmt);
 
         return spec;
@@ -2417,7 +2419,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         metadataProvider.getConfig().put(FeedActivityDetails.COLLECT_LOCATIONS,
                 StringUtils.join(bfs.getLocations(), ','));
 
-        JobSpecification jobSpec = rewriteCompileQuery(metadataProvider, bfs.getQuery(), csfs);
+        JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, bfs.getQuery(), csfs);
         FeedConnectionId feedConnectionId = new FeedConnectionId(bfs.getSubscriptionRequest().getReceivingFeedId(),
                 bfs.getSubscriptionRequest().getTargetDataset());
         String dataverse = feedConnectionId.getFeedId().getDataverse();
@@ -2560,7 +2562,7 @@ public class QueryTranslator extends AbstractLangTranslator implements IStatemen
         metadataProvider.setMetadataTxnContext(mdTxnCtx);
         MetadataLockManager.INSTANCE.queryBegin(activeDefaultDataverse, query.getDataverses(), query.getDatasets());
         try {
-            JobSpecification jobSpec = rewriteCompileQuery(metadataProvider, query, null);
+            JobSpecification jobSpec = rewriteCompileQuery(hcc, metadataProvider, query, null);
 
             MetadataManager.INSTANCE.commitTransaction(mdTxnCtx);
             bActiveTxn = false;

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/main/resources/asterix-build-configuration.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/resources/asterix-build-configuration.xml b/asterixdb/asterix-app/src/main/resources/asterix-build-configuration.xml
index 90dbe5f..bad6a35 100644
--- a/asterixdb/asterix-app/src/main/resources/asterix-build-configuration.xml
+++ b/asterixdb/asterix-app/src/main/resources/asterix-build-configuration.xml
@@ -67,10 +67,6 @@
     <value>160KB</value>
   </property>
   <property>
-    <name>compiler.pregelix.home</name>
-    <value>~/pregelix</value>
-  </property>
-  <property>
     <name>storage.buffercache.pagesize</name>
     <value>32KB</value>
     <description>The page size in bytes for pages in the buffer cache.

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/main/resources/asterix-build-configuration2.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/resources/asterix-build-configuration2.xml b/asterixdb/asterix-app/src/main/resources/asterix-build-configuration2.xml
new file mode 100644
index 0000000..7b8d274
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/resources/asterix-build-configuration2.xml
@@ -0,0 +1,116 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<asterixConfiguration xmlns="asterixconf">
+    <metadataNode>asterix_nc1</metadataNode>
+    <store>
+        <ncId>asterix_nc1</ncId>
+        <storeDirs>iodevice0,iodevice1</storeDirs>
+    </store>
+    <store>
+        <ncId>asterix_nc2</ncId>
+        <storeDirs>iodevice0,iodevice1</storeDirs>
+    </store>
+    <transactionLogDir>
+        <ncId>asterix_nc1</ncId>
+        <txnLogDirPath>target/txnLogDir/asterix_nc1</txnLogDirPath>
+    </transactionLogDir>
+    <transactionLogDir>
+        <ncId>asterix_nc2</ncId>
+        <txnLogDirPath>target/txnLogDir/asterix_nc2</txnLogDirPath>
+    </transactionLogDir>
+
+    <property>
+        <name>max.wait.active.cluster</name>
+        <value>60</value>
+        <description>Maximum wait (in seconds) for a cluster to be ACTIVE (all
+            nodes are available)
+            before a submitted query/statement can be
+            executed. (Default = 60 seconds)
+        </description>
+    </property>
+
+    <property>
+        <name>log.level</name>
+        <value>INFO</value>
+        <description>Log level for running tests/build</description>
+    </property>
+    <property>
+        <name>compiler.framesize</name>
+        <value>32KB</value>
+    </property>
+    <property>
+        <name>compiler.sortmemory</name>
+        <value>320KB</value>
+    </property>
+    <property>
+        <name>compiler.groupmemory</name>
+        <value>160KB</value>
+    </property>
+    <property>
+        <name>compiler.joinmemory</name>
+        <value>160KB</value>
+    </property>
+    <property>
+        <name>compiler.parallelism</name>
+        <value>-1</value>
+    </property>
+    <property>
+        <name>storage.buffercache.pagesize</name>
+        <value>32KB</value>
+        <description>The page size in bytes for pages in the buffer cache.
+            (Default = "128KB")
+        </description>
+    </property>
+    <property>
+        <name>storage.buffercache.size</name>
+        <value>48MB</value>
+        <description>The size of memory allocated to the disk buffer cache.
+            The value should be a multiple of the buffer cache page size
+            (Default = "512MB")
+        </description>
+    </property>
+    <property>
+        <name>storage.memorycomponent.numpages</name>
+        <value>8</value>
+        <description>The number of pages to allocate for a memory component.
+            This budget is shared by all the memory components of the primary
+            index and all its secondary indexes across all I/O devices on a node.
+            Note: in-memory components usually has fill factor of 75% since
+            the pages are 75% full and the remaining 25% is un-utilized. (Default = 256)
+        </description>
+    </property>
+    <property>
+        <name>plot.activate</name>
+        <value>false</value>
+        <description>Enabling plot of Algebricks plan to tmp folder. (Default = false)
+        </description>
+    </property>
+    <property>
+        <name>messaging.frame.size</name>
+        <value>4096</value>
+        <description>The frame size to be used for NC to NC messaging. (Default = 4kb)
+        </description>
+    </property>
+    <property>
+        <name>messaging.frame.count</name>
+        <value>512</value>
+        <description>Number of reusable frames for NC to NC messaging. (Default = 512)
+        </description>
+    </property>
+</asterixConfiguration>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/main/resources/asterix-build-configuration3.xml
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/main/resources/asterix-build-configuration3.xml b/asterixdb/asterix-app/src/main/resources/asterix-build-configuration3.xml
new file mode 100644
index 0000000..85b0cdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/main/resources/asterix-build-configuration3.xml
@@ -0,0 +1,116 @@
+<!--
+ ! Licensed to the Apache Software Foundation (ASF) under one
+ ! or more contributor license agreements.  See the NOTICE file
+ ! distributed with this work for additional information
+ ! regarding copyright ownership.  The ASF licenses this file
+ ! to you under the Apache License, Version 2.0 (the
+ ! "License"); you may not use this file except in compliance
+ ! with the License.  You may obtain a copy of the License at
+ !
+ !   http://www.apache.org/licenses/LICENSE-2.0
+ !
+ ! Unless required by applicable law or agreed to in writing,
+ ! software distributed under the License is distributed on an
+ ! "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ ! KIND, either express or implied.  See the License for the
+ ! specific language governing permissions and limitations
+ ! under the License.
+ !-->
+<asterixConfiguration xmlns="asterixconf">
+    <metadataNode>asterix_nc1</metadataNode>
+    <store>
+        <ncId>asterix_nc1</ncId>
+        <storeDirs>iodevice0,iodevice1</storeDirs>
+    </store>
+    <store>
+        <ncId>asterix_nc2</ncId>
+        <storeDirs>iodevice0,iodevice1</storeDirs>
+    </store>
+    <transactionLogDir>
+        <ncId>asterix_nc1</ncId>
+        <txnLogDirPath>target/txnLogDir/asterix_nc1</txnLogDirPath>
+    </transactionLogDir>
+    <transactionLogDir>
+        <ncId>asterix_nc2</ncId>
+        <txnLogDirPath>target/txnLogDir/asterix_nc2</txnLogDirPath>
+    </transactionLogDir>
+
+    <property>
+        <name>max.wait.active.cluster</name>
+        <value>60</value>
+        <description>Maximum wait (in seconds) for a cluster to be ACTIVE (all
+            nodes are available)
+            before a submitted query/statement can be
+            executed. (Default = 60 seconds)
+        </description>
+    </property>
+
+    <property>
+        <name>log.level</name>
+        <value>INFO</value>
+        <description>Log level for running tests/build</description>
+    </property>
+    <property>
+        <name>compiler.framesize</name>
+        <value>32KB</value>
+    </property>
+    <property>
+        <name>compiler.sortmemory</name>
+        <value>320KB</value>
+    </property>
+    <property>
+        <name>compiler.groupmemory</name>
+        <value>160KB</value>
+    </property>
+    <property>
+        <name>compiler.joinmemory</name>
+        <value>160KB</value>
+    </property>
+    <property>
+        <name>compiler.parallelism</name>
+        <value>3</value>
+    </property>
+    <property>
+        <name>storage.buffercache.pagesize</name>
+        <value>32KB</value>
+        <description>The page size in bytes for pages in the buffer cache.
+            (Default = "128KB")
+        </description>
+    </property>
+    <property>
+        <name>storage.buffercache.size</name>
+        <value>48MB</value>
+        <description>The size of memory allocated to the disk buffer cache.
+            The value should be a multiple of the buffer cache page size
+            (Default = "512MB")
+        </description>
+    </property>
+    <property>
+        <name>storage.memorycomponent.numpages</name>
+        <value>8</value>
+        <description>The number of pages to allocate for a memory component.
+            This budget is shared by all the memory components of the primary
+            index and all its secondary indexes across all I/O devices on a node.
+            Note: in-memory components usually has fill factor of 75% since
+            the pages are 75% full and the remaining 25% is un-utilized. (Default = 256)
+        </description>
+    </property>
+    <property>
+        <name>plot.activate</name>
+        <value>false</value>
+        <description>Enabling plot of Algebricks plan to tmp folder. (Default = false)
+        </description>
+    </property>
+    <property>
+        <name>messaging.frame.size</name>
+        <value>4096</value>
+        <description>The frame size to be used for NC to NC messaging. (Default = 4kb)
+        </description>
+    </property>
+    <property>
+        <name>messaging.frame.count</name>
+        <value>512</value>
+        <description>Number of reusable frames for NC to NC messaging. (Default = 512)
+        </description>
+    </property>
+</asterixConfiguration>

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java
new file mode 100644
index 0000000..6256857
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/common/APIFrameworkTest.java
@@ -0,0 +1,134 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT 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.api.common;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.asterix.compiler.provider.ILangCompilationProvider;
+import org.apache.hyracks.algebricks.common.constraints.AlgebricksAbsolutePartitionConstraint;
+import org.apache.hyracks.api.client.IClusterInfoCollector;
+import org.apache.hyracks.api.client.NodeControllerInfo;
+import org.junit.Assert;
+import org.junit.Test;
+
+import junit.extensions.PA;
+
+public class APIFrameworkTest {
+
+    @Test
+    public void testGetComputationLocations() throws Exception {
+        IClusterInfoCollector clusterInfoCollector = mock(IClusterInfoCollector.class);
+
+        // Constructs mocked cluster nodes.
+        Map<String, NodeControllerInfo> map = new HashMap<>();
+        NodeControllerInfo nc1Info = mock(NodeControllerInfo.class);
+        when(nc1Info.getNumCores()).thenReturn(4);
+        NodeControllerInfo nc2Info = mock(NodeControllerInfo.class);
+        when(nc2Info.getNumCores()).thenReturn(4);
+        String nc1 = "nc1";
+        String nc2 = "nc2";
+        map.put(nc1, nc1Info);
+        map.put(nc2, nc2Info);
+        when(clusterInfoCollector.getNodeControllerInfos()).thenReturn(map);
+
+        // Creates an APIFramework.
+        APIFramework apiFramework = new APIFramework(mock(ILangCompilationProvider.class));
+
+        // Tests odd number parallelism.
+        AlgebricksAbsolutePartitionConstraint loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(
+                apiFramework, "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)",
+                clusterInfoCollector, 5);
+        int nc1Count = 0, nc2Count = 0;
+        String[] partitions = loc.getLocations();
+        for (String partition : partitions) {
+            if (partition.equals(nc1)) {
+                ++nc1Count;
+            }
+            if (partition.equals(nc2)) {
+                ++nc2Count;
+            }
+        }
+        Assert.assertTrue(nc1Count > 0);
+        Assert.assertTrue(nc2Count > 0);
+        Assert.assertTrue(Math.abs(nc1Count - nc2Count) == 1); // Tests load balance.
+        Assert.assertTrue(partitions.length == 5);
+
+        // Tests even number parallelism.
+        loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
+                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, 8);
+        nc1Count = 0;
+        nc2Count = 0;
+        partitions = loc.getLocations();
+        for (String partition : partitions) {
+            if (partition.equals(nc1)) {
+                ++nc1Count;
+            }
+            if (partition.equals(nc2)) {
+                ++nc2Count;
+            }
+        }
+        Assert.assertTrue(nc1Count > 0);
+        Assert.assertTrue(nc2Count > 0);
+        Assert.assertTrue(Math.abs(nc1Count - nc2Count) == 0); // Tests load balance.
+        // The maximum parallelism cannot be beyond n *(#core-1), where n is the number of NCs and #core is the number
+        // of cores per NC.
+        Assert.assertTrue(partitions.length == 6);
+
+        // Tests the case when parallelism is one.
+        loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
+                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, 1);
+        Assert.assertTrue(loc.getLocations().length == 1);
+
+        // Tests the case when parallelism is a negative.
+        // In this case, the compiler has no idea and falls back to the default setting where all possible cores
+        // are used.
+        loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
+                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector,
+                -100);
+        Assert.assertTrue(loc.getLocations().length == 6);
+
+        // Tests the case when parallelism is -1.
+        // In this case, the compiler has no idea and falls back to the default setting where all possible cores
+        // are used.
+        loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
+                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, -1);
+        Assert.assertTrue(loc.getLocations().length == 6);
+
+        // Tests the case when parallelism is zero.
+        // In this case, the compiler has no idea and falls back to the default setting where all possible cores
+        // are used.
+        loc = (AlgebricksAbsolutePartitionConstraint) PA.invokeMethod(apiFramework,
+                "getComputationLocations(" + IClusterInfoCollector.class.getName() + ",int)", clusterInfoCollector, 0);
+        Assert.assertTrue(loc.getLocations().length == 6);
+
+        // Verifies the number of calls on clusterInfoCollector.getNodeControllerInfos() in
+        // APIFramework.getComputationLocations(...).
+        verify(clusterInfoCollector, times(6)).getNodeControllerInfos();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
index 71b2df5..fdcbad5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/ConnectorAPIServletTest.java
@@ -44,7 +44,7 @@ import org.apache.asterix.om.types.ARecordType;
 import org.apache.asterix.om.types.BuiltinType;
 import org.apache.asterix.om.types.IAType;
 import org.apache.asterix.om.util.JSONDeserializerForTypes;
-import org.apache.asterix.test.runtime.ExecutionTest;
+import org.apache.asterix.test.runtime.SqlppExecutionTest;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.apache.hyracks.api.client.NodeControllerInfo;
 import org.apache.hyracks.api.comm.NetworkAddress;
@@ -63,7 +63,7 @@ public class ConnectorAPIServletTest {
     @Test
     public void testGet() throws Exception {
         // Starts test asterixdb cluster.
-        ExecutionTest.setUp();
+        SqlppExecutionTest.setUp();
 
         // Configures a test connector api servlet.
         ConnectorAPIServlet servlet = spy(new ConnectorAPIServlet());
@@ -116,7 +116,7 @@ public class ConnectorAPIServletTest {
         Assert.assertTrue(path.endsWith("Metadata/Dataset_idx_Dataset"));
 
         // Tears down the asterixdb cluster.
-        ExecutionTest.tearDown();
+        SqlppExecutionTest.tearDown();
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java
index df7c151..2209fc5 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/api/http/servlet/VersionAPIServletTest.java
@@ -19,6 +19,12 @@
 
 package org.apache.asterix.api.http.servlet;
 
+import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
+import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.spy;
+import static org.mockito.Mockito.when;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.InputStreamReader;
@@ -33,25 +39,19 @@ import javax.servlet.http.HttpServletResponse;
 
 import org.apache.asterix.common.config.BuildProperties;
 import org.apache.asterix.runtime.util.AppContextInfo;
-import org.apache.asterix.test.runtime.ExecutionTest;
+import org.apache.asterix.test.runtime.SqlppExecutionTest;
 import org.apache.hyracks.api.client.IHyracksClientConnection;
 import org.json.JSONObject;
 import org.json.JSONTokener;
 import org.junit.Assert;
 import org.junit.Test;
 
-import static org.apache.asterix.api.http.servlet.ServletConstants.ASTERIX_BUILD_PROP_ATTR;
-import static org.apache.asterix.api.http.servlet.ServletConstants.HYRACKS_CONNECTION_ATTR;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.spy;
-import static org.mockito.Mockito.when;
-
 public class VersionAPIServletTest {
 
     @Test
     public void testGet() throws Exception {
         // Starts test asterixdb cluster.
-        ExecutionTest.setUp();
+        SqlppExecutionTest.setUp();
 
         // Configures a test version api servlet.
         VersionAPIServlet servlet = spy(new VersionAPIServlet());
@@ -112,6 +112,6 @@ public class VersionAPIServletTest {
         Assert.assertEquals(actualResponse.toString(), expectedResponse.toString());
 
         // Tears down the asterixdb cluster.
-        ExecutionTest.tearDown();
+        SqlppExecutionTest.tearDown();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
new file mode 100644
index 0000000..5afcb55
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionFullParallelismIT.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the AQL runtime tests with full parallelism on node controllers.
+ */
+@RunWith(Parameterized.class)
+public class AqlExecutionFullParallelismIT {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration2.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "AqlExecutionFullParallelismIT {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only.xml", "testsuite.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public AqlExecutionFullParallelismIT(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
new file mode 100644
index 0000000..da826a3
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionLessParallelismIT.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the AQL runtime tests with less parallelism on node controllers than using all the cores.
+ */
+@RunWith(Parameterized.class)
+public class AqlExecutionLessParallelismIT {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration3.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "AqlExecutionLessParallelismIT {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only.xml", "testsuite.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public AqlExecutionLessParallelismIT(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionTest.java
new file mode 100644
index 0000000..4588d87
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/AqlExecutionTest.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the AQL runtime tests with the storage parallelism.
+ */
+@RunWith(Parameterized.class)
+public class AqlExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "AqlExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only.xml", "testsuite.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public AqlExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionFullParallelismTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionFullParallelismTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionFullParallelismTest.java
new file mode 100644
index 0000000..2f6aa1f
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionFullParallelismTest.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the cluster state runtime tests with with full parallelism on node controllers.
+ */
+@RunWith(Parameterized.class)
+public class ClusterStateExecutionFullParallelismTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration2.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "ClusterStateExecutionFullParallelismTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_cluster_state.xml", "cluster_state_full.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public ClusterStateExecutionFullParallelismTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionLessParallelismTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionLessParallelismTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionLessParallelismTest.java
new file mode 100644
index 0000000..73b8cdb
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionLessParallelismTest.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the cluster state tests with less parallelism on node controllers than using all the cores.
+ */
+@RunWith(Parameterized.class)
+public class ClusterStateExecutionLessParallelismTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration3.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "ClusterStateExecutionLessParallelismTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_cluster_state.xml", "cluster_state_less.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public ClusterStateExecutionLessParallelismTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionTest.java
new file mode 100644
index 0000000..14551db
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ClusterStateExecutionTest.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the cluster state runtime tests with the storage parallelism.
+ */
+@RunWith(Parameterized.class)
+public class ClusterStateExecutionTest {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "ClusterStateExecutionTest {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_cluster_state.xml", "cluster_state.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public ClusterStateExecutionTest(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
deleted file mode 100644
index a733645..0000000
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTest.java
+++ /dev/null
@@ -1,139 +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.test.runtime;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-import java.util.logging.Logger;
-
-import org.apache.asterix.app.external.TestLibrarian;
-import org.apache.asterix.common.config.TransactionProperties;
-import org.apache.asterix.common.library.ILibraryManager;
-import org.apache.asterix.test.aql.TestExecutor;
-import org.apache.asterix.testframework.context.TestCaseContext;
-import org.apache.commons.lang3.StringUtils;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.junit.runners.Parameterized.Parameters;
-
-/**
- * Runs the runtime test cases under 'asterix-app/src/test/resources/runtimets'.
- */
-@RunWith(Parameterized.class)
-public class ExecutionTest {
-
-    protected static final Logger LOGGER = Logger.getLogger(ExecutionTest.class.getName());
-
-    protected static final String PATH_ACTUAL = "target" + File.separator + "rttest" + File.separator;
-    protected static final String PATH_BASE = StringUtils.join(new String[] { "src", "test", "resources", "runtimets" },
-            File.separator);
-
-    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration.xml";
-    protected static final List<String> badTestCases = new ArrayList<>();
-    protected static TransactionProperties txnProperties;
-    protected static final TestExecutor testExecutor = new TestExecutor();
-    private static final boolean cleanupOnStart = true;
-    private static final boolean cleanupOnStop = true;
-    private static TestLibrarian librarian;
-    private static final int repeat = Integer.getInteger("test.repeat", 1);
-
-    @BeforeClass
-    public static void setUp() throws Exception {
-        try {
-            File outdir = new File(PATH_ACTUAL);
-            outdir.mkdirs();
-            // remove library directory
-            TestLibrarian.removeLibraryDir();
-            List<ILibraryManager> libraryManagers = ExecutionTestUtil.setUp(cleanupOnStart);
-            librarian = new TestLibrarian(libraryManagers);
-            testExecutor.setLibrarian(librarian);
-            if (repeat != 1) {
-                System.out.println("FYI: each test will be run " + repeat + " times.");
-            }
-        } catch (Throwable th) {
-            th.printStackTrace();
-            throw th;
-        }
-    }
-
-    @AfterClass
-    public static void tearDown() throws Exception {
-        // remove library directory
-        TestLibrarian.removeLibraryDir();
-        ExecutionTestUtil.tearDown(cleanupOnStop);
-        if (!badTestCases.isEmpty()) {
-            System.out.println("The following test cases left some data");
-            for (String testCase : badTestCases) {
-                System.out.println(testCase);
-            }
-        }
-    }
-
-    @Parameters(name = "ExecutionTest {index}: {0}")
-    public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.ONLY_TESTSUITE_XML_NAME);
-        if (testArgs.size() == 0) {
-            testArgs = buildTestsInXml(TestCaseContext.DEFAULT_TESTSUITE_XML_NAME);
-        }
-        return testArgs;
-    }
-
-    protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
-        Collection<Object[]> testArgs = new ArrayList<Object[]>();
-        TestCaseContext.Builder b = new TestCaseContext.Builder();
-        for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
-            testArgs.add(new Object[] { ctx });
-        }
-        return testArgs;
-
-    }
-
-    protected TestCaseContext tcCtx;
-
-    public ExecutionTest(TestCaseContext tcCtx) {
-        this.tcCtx = tcCtx;
-    }
-
-    @Test
-    public void test() throws Exception {
-        int repeat = ExecutionTest.repeat * tcCtx.getRepeat();
-        try {
-            for (int i = 1; i <= repeat; i++) {
-                if (repeat > 1) {
-                    System.err.print("[" + i + "/" + repeat + "] ");
-                }
-                librarian.cleanup();
-                testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false, ExecutionTestUtil.FailedGroup);
-                try {
-                    testExecutor.cleanup(tcCtx.toString(), badTestCases);
-                } catch (Throwable th) {
-                    th.printStackTrace();
-                    throw th;
-                }
-            }
-        } finally {
-            System.err.flush();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
index b4bbc36..f488244 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/ExecutionTestUtil.java
@@ -38,7 +38,7 @@ import org.apache.hyracks.control.nc.NodeControllerService;
 
 public class ExecutionTestUtil {
 
-    protected static final Logger LOGGER = Logger.getLogger(ExecutionTest.class.getName());
+    protected static final Logger LOGGER = Logger.getLogger(ExecutionTestUtil.class.getName());
 
     protected static final String PATH_ACTUAL = "rttest" + File.separator;
 

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
new file mode 100644
index 0000000..29534f1
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/LangExecutionUtil.java
@@ -0,0 +1,115 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.asterix.app.external.TestLibrarian;
+import org.apache.asterix.common.library.ILibraryManager;
+import org.apache.asterix.test.aql.TestExecutor;
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+/**
+ * Utils for running SQL++ or AQL runtime tests.
+ */
+@RunWith(Parameterized.class)
+public class LangExecutionUtil {
+
+    private static final String PATH_ACTUAL = "target" + File.separator + "rttest" + File.separator;
+    private static final String PATH_BASE = StringUtils.join(new String[] { "src", "test", "resources", "runtimets" },
+            File.separator);
+
+    private static final boolean cleanupOnStart = true;
+    private static final boolean cleanupOnStop = true;
+    private static final List<String> badTestCases = new ArrayList<>();
+    private static final TestExecutor testExecutor = new TestExecutor();
+
+    private static TestLibrarian librarian;
+    private static final int repeat = Integer.getInteger("test.repeat", 1);
+
+    public static void setUp(String configFile) throws Exception {
+        File outdir = new File(PATH_ACTUAL);
+        outdir.mkdirs();
+        List<ILibraryManager> libraryManagers = ExecutionTestUtil.setUp(cleanupOnStart, configFile);
+        TestLibrarian.removeLibraryDir();
+        librarian = new TestLibrarian(libraryManagers);
+        testExecutor.setLibrarian(librarian);
+        if (repeat != 1) {
+            System.out.println("FYI: each test will be run " + repeat + " times.");
+        }
+    }
+
+    public static void tearDown() throws Exception {
+        TestLibrarian.removeLibraryDir();
+        ExecutionTestUtil.tearDown(cleanupOnStop);
+        ExecutionTestUtil.integrationUtil.removeTestStorageFiles();
+        if (!badTestCases.isEmpty()) {
+            System.out.println("The following test cases left some data");
+            for (String testCase : badTestCases) {
+                System.out.println(testCase);
+            }
+        }
+    }
+
+    public static Collection<Object[]> tests(String onlyFilePath, String suiteFilePath) throws Exception {
+        Collection<Object[]> testArgs = buildTestsInXml(onlyFilePath);
+        if (testArgs.size() == 0) {
+            testArgs = buildTestsInXml(suiteFilePath);
+        }
+        return testArgs;
+    }
+
+    protected static Collection<Object[]> buildTestsInXml(String xmlfile) throws Exception {
+        Collection<Object[]> testArgs = new ArrayList<>();
+        TestCaseContext.Builder b = new TestCaseContext.Builder();
+        for (TestCaseContext ctx : b.build(new File(PATH_BASE), xmlfile)) {
+            testArgs.add(new Object[] { ctx });
+        }
+        return testArgs;
+    }
+
+    public static void test(TestCaseContext tcCtx) throws Exception {
+        int repeat = LangExecutionUtil.repeat * tcCtx.getRepeat();
+        try {
+            for (int i = 1; i <= repeat; i++) {
+                if (repeat > 1) {
+                    System.err.print("[" + i + "/" + repeat + "] ");
+                }
+                librarian.cleanup();
+                testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false, ExecutionTestUtil.FailedGroup);
+                try {
+                    testExecutor.cleanup(tcCtx.toString(), badTestCases);
+                } catch (Throwable th) {
+                    th.printStackTrace();
+                    throw th;
+                }
+            }
+        } finally {
+            System.err.flush();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RepeatedTest.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RepeatedTest.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RepeatedTest.java
index 7d39b31..b7e32ab 100644
--- a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RepeatedTest.java
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/RepeatedTest.java
@@ -18,6 +18,8 @@
  */
 package org.apache.asterix.test.runtime;
 
+import static org.apache.asterix.test.runtime.LangExecutionUtil.buildTestsInXml;
+
 import java.lang.annotation.Retention;
 import java.lang.annotation.RetentionPolicy;
 import java.lang.annotation.Target;
@@ -79,14 +81,13 @@ class RepeatRule implements MethodRule {
 }
 
 @RunWith(Parameterized.class)
-public class RepeatedTest extends ExecutionTest {
+public class RepeatedTest extends SqlppExecutionTest {
 
     private int count;
 
     @Parameters(name = "RepeatedTest {index}: {0}")
     public static Collection<Object[]> tests() throws Exception {
-        Collection<Object[]> testArgs = buildTestsInXml(TestCaseContext.DEFAULT_REPEATED_TESTSUITE_XML_NAME);
-        return testArgs;
+        return LangExecutionUtil.buildTestsInXml(TestCaseContext.DEFAULT_REPEATED_TESTSUITE_XML_NAME);
     }
 
     public RepeatedTest(TestCaseContext tcCtx) {
@@ -102,6 +103,6 @@ public class RepeatedTest extends ExecutionTest {
     @Repeat(times = 100)
     public void test() throws Exception {
         System.err.println("***** Test Count: " + (++count) + " ******");
-        testExecutor.executeTest(PATH_ACTUAL, tcCtx, null, false);
+        super.test();
     }
 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionFullParallelismIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionFullParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionFullParallelismIT.java
new file mode 100644
index 0000000..62654ca
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionFullParallelismIT.java
@@ -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.
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ runtime tests with full parallelism on node controllers.
+ */
+@RunWith(Parameterized.class)
+public class SqlppExecutionFullParallelismIT {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration2.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppExecutionFullParallelismIT {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_sqlpp.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppExecutionFullParallelismIT(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionLessParallelismIT.java
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionLessParallelismIT.java b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionLessParallelismIT.java
new file mode 100644
index 0000000..1b22d71
--- /dev/null
+++ b/asterixdb/asterix-app/src/test/java/org/apache/asterix/test/runtime/SqlppExecutionLessParallelismIT.java
@@ -0,0 +1,65 @@
+/*
+ *
+ *  * Licensed to the Apache Software Foundation (ASF) under one
+ *  * or more contributor license agreements.  See the NOTICE file
+ *  * distributed with this work for additional information
+ *  * regarding copyright ownership.  The ASF licenses this file
+ *  * to you under the Apache License, Version 2.0 (the
+ *  * "License"); you may not use this file except in compliance
+ *  * with the License.  You may obtain a copy of the License at
+ *  *
+ *  *   http://www.apache.org/licenses/LICENSE-2.0
+ *  *
+ *  * Unless required by applicable law or agreed to in writing,
+ *  * software distributed under the License is distributed on an
+ *  * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ *  * KIND, either express or implied.  See the License for the
+ *  * specific language governing permissions and limitations
+ *  * under the License.
+ *
+ */
+package org.apache.asterix.test.runtime;
+
+import java.util.Collection;
+
+import org.apache.asterix.testframework.context.TestCaseContext;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+/**
+ * Runs the SQL++ runtime tests with less parallelism on node controllers than using all the cores.
+ */
+@RunWith(Parameterized.class)
+public class SqlppExecutionLessParallelismIT {
+    protected static final String TEST_CONFIG_FILE_NAME = "asterix-build-configuration3.xml";
+
+    @BeforeClass
+    public static void setUp() throws Exception {
+        LangExecutionUtil.setUp(TEST_CONFIG_FILE_NAME);
+    }
+
+    @AfterClass
+    public static void tearDown() throws Exception {
+        LangExecutionUtil.tearDown();
+    }
+
+    @Parameters(name = "SqlppExecutionLessParallelismIT {index}: {0}")
+    public static Collection<Object[]> tests() throws Exception {
+        return LangExecutionUtil.tests("only_sqlpp.xml", "testsuite_sqlpp.xml");
+    }
+
+    protected TestCaseContext tcCtx;
+
+    public SqlppExecutionLessParallelismIT(TestCaseContext tcCtx) {
+        this.tcCtx = tcCtx;
+    }
+
+    @Test
+    public void test() throws Exception {
+        LangExecutionUtil.test(tcCtx);
+    }
+}


[4/8] asterixdb git commit: Supports flexible degree of parallelism.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.3.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.3.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.3.adm
index 94e383c..74e1bb7 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.3.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at00/at00.3.adm
@@ -1,105 +1,456 @@
+{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15" }
+{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30" }
+{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17" }
+{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23" }
+{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01" }
+{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18" }
+{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25" }
+{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24" }
+{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03" }
+{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02" }
+{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03" }
+{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18" }
+{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02" }
+{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14" }
+{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06" }
 { "partkey": 6, "pid": 1, "shipdate": "1992-04-05" }
 { "partkey": 6, "pid": 2, "shipdate": "1992-04-25" }
 { "partkey": 6, "pid": 3, "shipdate": "1992-04-29" }
+{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11" }
+{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25" }
+{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25" }
+{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15" }
+{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13" }
+{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29" }
+{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30" }
+{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13" }
+{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01" }
 { "partkey": 11, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 11, "pid": 2, "shipdate": "1992-07-20" }
 { "partkey": 11, "pid": 3, "shipdate": "1992-08-03" }
 { "partkey": 12, "pid": 1, "shipdate": "1992-07-04" }
 { "partkey": 12, "pid": 2, "shipdate": "1992-07-17" }
 { "partkey": 12, "pid": 3, "shipdate": "1992-09-02" }
+{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01" }
+{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26" }
+{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04" }
 { "partkey": 14, "pid": 1, "shipdate": "1992-07-17" }
 { "partkey": 14, "pid": 2, "shipdate": "1992-11-30" }
 { "partkey": 14, "pid": 3, "shipdate": "1993-05-10" }
+{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18" }
+{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24" }
+{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14" }
+{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11" }
+{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25" }
+{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17" }
+{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23" }
+{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01" }
+{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06" }
+{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21" }
+{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21" }
+{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19" }
+{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21" }
+{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22" }
+{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15" }
+{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29" }
+{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18" }
 { "partkey": 21, "pid": 1, "shipdate": "1992-07-31" }
 { "partkey": 21, "pid": 2, "shipdate": "1992-09-09" }
 { "partkey": 21, "pid": 3, "shipdate": "1993-01-09" }
+{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21" }
+{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25" }
+{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20" }
 { "partkey": 23, "pid": 1, "shipdate": "1992-04-04" }
 { "partkey": 23, "pid": 2, "shipdate": "1992-06-19" }
 { "partkey": 23, "pid": 3, "shipdate": "1992-06-29" }
+{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06" }
+{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08" }
+{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04" }
+{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23" }
+{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01" }
 { "partkey": 26, "pid": 1, "shipdate": "1992-02-23" }
 { "partkey": 26, "pid": 2, "shipdate": "1992-05-09" }
 { "partkey": 26, "pid": 3, "shipdate": "1993-01-04" }
+{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14" }
+{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17" }
+{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16" }
+{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13" }
+{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04" }
+{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25" }
+{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01" }
+{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25" }
 { "partkey": 30, "pid": 1, "shipdate": "1992-04-10" }
 { "partkey": 30, "pid": 2, "shipdate": "1992-05-18" }
 { "partkey": 30, "pid": 3, "shipdate": "1992-05-21" }
+{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14" }
+{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24" }
+{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29" }
+{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22" }
+{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25" }
+{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07" }
 { "partkey": 33, "pid": 1, "shipdate": "1992-03-22" }
 { "partkey": 33, "pid": 2, "shipdate": "1993-02-17" }
 { "partkey": 33, "pid": 3, "shipdate": "1993-02-21" }
+{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03" }
+{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20" }
+{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23" }
+{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11" }
+{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06" }
+{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26" }
+{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26" }
+{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03" }
+{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06" }
+{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30" }
+{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03" }
+{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31" }
 { "partkey": 38, "pid": 1, "shipdate": "1992-04-06" }
 { "partkey": 38, "pid": 2, "shipdate": "1992-04-15" }
 { "partkey": 38, "pid": 3, "shipdate": "1992-08-27" }
+{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26" }
+{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12" }
+{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15" }
+{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07" }
+{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03" }
+{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13" }
+{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18" }
+{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13" }
+{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23" }
+{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04" }
+{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12" }
+{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30" }
+{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28" }
+{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14" }
+{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11" }
+{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29" }
 { "partkey": 45, "pid": 1, "shipdate": "1992-07-16" }
 { "partkey": 45, "pid": 2, "shipdate": "1993-06-24" }
 { "partkey": 45, "pid": 3, "shipdate": "1993-09-15" }
+{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28" }
+{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08" }
+{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21" }
 { "partkey": 47, "pid": 1, "shipdate": "1992-03-11" }
 { "partkey": 47, "pid": 2, "shipdate": "1993-05-30" }
 { "partkey": 47, "pid": 3, "shipdate": "1993-06-06" }
+{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10" }
+{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03" }
+{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15" }
 { "partkey": 49, "pid": 1, "shipdate": "1992-04-29" }
 { "partkey": 49, "pid": 2, "shipdate": "1992-06-14" }
 { "partkey": 49, "pid": 3, "shipdate": "1992-08-13" }
+{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22" }
+{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31" }
+{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23" }
 { "partkey": 51, "pid": 1, "shipdate": "1992-03-11" }
 { "partkey": 51, "pid": 2, "shipdate": "1992-05-15" }
 { "partkey": 51, "pid": 3, "shipdate": "1992-05-17" }
+{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31" }
+{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03" }
+{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21" }
+{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14" }
+{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22" }
+{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04" }
+{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07" }
+{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01" }
+{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24" }
+{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11" }
+{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17" }
+{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02" }
+{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18" }
+{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16" }
+{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06" }
+{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21" }
 { "partkey": 58, "pid": 1, "shipdate": "1992-05-16" }
 { "partkey": 58, "pid": 2, "shipdate": "1992-10-30" }
 { "partkey": 58, "pid": 3, "shipdate": "1993-04-10" }
+{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09" }
+{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17" }
+{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12" }
 { "partkey": 60, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 60, "pid": 2, "shipdate": "1992-07-01" }
 { "partkey": 60, "pid": 3, "shipdate": "1992-07-15" }
+{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14" }
+{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15" }
+{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29" }
+{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26" }
+{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19" }
+{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07" }
+{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15" }
+{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07" }
+{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13" }
+{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14" }
+{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10" }
+{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02" }
+{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14" }
+{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26" }
+{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11" }
+{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10" }
+{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13" }
+{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08" }
+{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03" }
+{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13" }
+{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08" }
+{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22" }
+{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31" }
+{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05" }
+{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01" }
 { "partkey": 70, "pid": 1, "shipdate": "1992-04-06" }
 { "partkey": 70, "pid": 2, "shipdate": "1992-06-11" }
 { "partkey": 70, "pid": 3, "shipdate": "1992-06-25" }
+{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10" }
+{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10" }
+{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28" }
 { "partkey": 72, "pid": 1, "shipdate": "1992-09-16" }
 { "partkey": 72, "pid": 2, "shipdate": "1992-10-02" }
 { "partkey": 72, "pid": 3, "shipdate": "1992-10-17" }
+{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08" }
+{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16" }
+{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02" }
+{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21" }
+{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22" }
+{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21" }
+{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27" }
+{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12" }
+{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19" }
+{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22" }
+{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19" }
+{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12" }
 { "partkey": 77, "pid": 1, "shipdate": "1992-08-18" }
 { "partkey": 77, "pid": 2, "shipdate": "1992-12-23" }
 { "partkey": 77, "pid": 3, "shipdate": "1993-06-19" }
+{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04" }
+{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04" }
+{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06" }
+{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05" }
+{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10" }
+{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08" }
+{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18" }
+{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02" }
+{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07" }
+{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11" }
+{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22" }
+{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30" }
+{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17" }
+{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18" }
+{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11" }
+{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09" }
+{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04" }
+{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21" }
+{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08" }
+{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15" }
+{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20" }
+{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28" }
+{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28" }
+{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27" }
+{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25" }
+{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18" }
+{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01" }
+{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30" }
+{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02" }
+{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06" }
 { "partkey": 88, "pid": 1, "shipdate": "1992-04-24" }
 { "partkey": 88, "pid": 2, "shipdate": "1992-06-26" }
 { "partkey": 88, "pid": 3, "shipdate": "1992-12-18" }
 { "partkey": 89, "pid": 1, "shipdate": "1992-04-18" }
 { "partkey": 89, "pid": 2, "shipdate": "1992-04-19" }
 { "partkey": 89, "pid": 3, "shipdate": "1992-05-27" }
+{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25" }
+{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07" }
+{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21" }
+{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22" }
+{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21" }
+{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03" }
 { "partkey": 92, "pid": 1, "shipdate": "1992-02-11" }
 { "partkey": 92, "pid": 2, "shipdate": "1992-09-30" }
 { "partkey": 92, "pid": 3, "shipdate": "1993-01-04" }
 { "partkey": 93, "pid": 1, "shipdate": "1992-05-28" }
 { "partkey": 93, "pid": 2, "shipdate": "1992-06-24" }
 { "partkey": 93, "pid": 3, "shipdate": "1992-09-11" }
+{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20" }
+{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03" }
+{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26" }
+{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24" }
+{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14" }
+{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17" }
+{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26" }
+{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25" }
+{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27" }
+{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22" }
+{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21" }
+{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06" }
+{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09" }
+{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09" }
+{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01" }
+{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18" }
+{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09" }
+{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24" }
+{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24" }
+{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18" }
+{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17" }
+{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27" }
+{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28" }
+{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19" }
+{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21" }
+{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25" }
 { "partkey": 103, "pid": 1, "shipdate": "1992-03-28" }
 { "partkey": 103, "pid": 2, "shipdate": "1992-05-08" }
 { "partkey": 103, "pid": 3, "shipdate": "1992-07-11" }
+{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17" }
+{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08" }
+{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22" }
 { "partkey": 105, "pid": 1, "shipdate": "1992-02-14" }
 { "partkey": 105, "pid": 2, "shipdate": "1992-06-01" }
 { "partkey": 105, "pid": 3, "shipdate": "1992-07-14" }
+{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09" }
+{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31" }
+{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02" }
+{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22" }
+{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30" }
+{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05" }
+{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28" }
+{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01" }
+{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07" }
 { "partkey": 109, "pid": 1, "shipdate": "1992-06-06" }
 { "partkey": 109, "pid": 2, "shipdate": "1992-11-20" }
 { "partkey": 109, "pid": 3, "shipdate": "1992-12-23" }
-{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13" }
-{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29" }
-{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17" }
-{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27" }
-{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02" }
-{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15" }
+{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18" }
+{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01" }
+{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01" }
+{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28" }
+{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13" }
+{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13" }
+{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09" }
+{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15" }
+{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08" }
+{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13" }
+{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25" }
+{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19" }
+{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22" }
+{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22" }
+{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13" }
+{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29" }
+{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17" }
+{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22" }
+{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17" }
+{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24" }
+{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04" }
+{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18" }
+{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10" }
+{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18" }
+{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27" }
+{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02" }
+{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08" }
+{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27" }
+{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07" }
+{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23" }
+{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29" }
+{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23" }
+{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09" }
+{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23" }
+{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12" }
+{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09" }
+{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05" }
+{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22" }
+{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15" }
+{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09" }
+{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13" }
+{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15" }
 { "partkey": 125, "pid": 2, "shipdate": "1992-03-29" }
 { "partkey": 125, "pid": 3, "shipdate": "1992-05-24" }
 { "partkey": 126, "pid": 1, "shipdate": "1992-07-28" }
 { "partkey": 126, "pid": 2, "shipdate": "1992-08-28" }
 { "partkey": 126, "pid": 3, "shipdate": "1992-09-06" }
+{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04" }
+{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02" }
+{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13" }
+{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05" }
+{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02" }
+{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24" }
+{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31" }
+{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28" }
+{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15" }
+{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03" }
+{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23" }
+{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20" }
+{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27" }
+{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03" }
+{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17" }
+{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14" }
+{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06" }
+{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08" }
+{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17" }
+{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18" }
+{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17" }
+{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20" }
+{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29" }
 { "partkey": 135, "pid": 1, "shipdate": "1992-05-02" }
 { "partkey": 135, "pid": 2, "shipdate": "1992-05-11" }
 { "partkey": 135, "pid": 3, "shipdate": "1992-05-29" }
+{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19" }
+{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21" }
+{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07" }
+{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23" }
+{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05" }
+{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12" }
 { "partkey": 138, "pid": 1, "shipdate": "1992-06-20" }
 { "partkey": 138, "pid": 2, "shipdate": "1992-11-21" }
 { "partkey": 138, "pid": 3, "shipdate": "1993-02-28" }
+{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28" }
+{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12" }
+{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20" }
+{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27" }
+{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03" }
 { "partkey": 141, "pid": 1, "shipdate": "1992-01-13" }
 { "partkey": 141, "pid": 2, "shipdate": "1992-02-01" }
 { "partkey": 141, "pid": 3, "shipdate": "1992-06-22" }
+{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14" }
+{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14" }
+{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11" }
+{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17" }
+{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01" }
+{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05" }
+{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05" }
+{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25" }
+{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17" }
+{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25" }
+{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16" }
+{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25" }
+{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21" }
+{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21" }
+{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02" }
 { "partkey": 147, "pid": 1, "shipdate": "1992-06-10" }
 { "partkey": 147, "pid": 2, "shipdate": "1992-09-04" }
 { "partkey": 147, "pid": 3, "shipdate": "1992-12-03" }
+{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15" }
+{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27" }
+{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22" }
+{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22" }
+{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29" }
+{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01" }
+{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02" }
+{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25" }
+{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26" }
+{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30" }
+{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19" }
 { "partkey": 152, "pid": 1, "shipdate": "1992-06-23" }
 { "partkey": 152, "pid": 2, "shipdate": "1993-05-19" }
 { "partkey": 152, "pid": 3, "shipdate": "1993-10-31" }
@@ -109,18 +460,75 @@
 { "partkey": 154, "pid": 1, "shipdate": "1992-02-18" }
 { "partkey": 154, "pid": 2, "shipdate": "1992-02-20" }
 { "partkey": 154, "pid": 3, "shipdate": "1992-05-14" }
+{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28" }
+{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14" }
 { "partkey": 156, "pid": 1, "shipdate": "1992-04-24" }
 { "partkey": 156, "pid": 2, "shipdate": "1992-06-17" }
 { "partkey": 156, "pid": 3, "shipdate": "1992-07-01" }
 { "partkey": 157, "pid": 1, "shipdate": "1992-07-26" }
 { "partkey": 157, "pid": 2, "shipdate": "1992-08-11" }
 { "partkey": 157, "pid": 3, "shipdate": "1992-08-25" }
+{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01" }
+{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29" }
+{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18" }
+{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03" }
+{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10" }
+{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07" }
+{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04" }
+{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18" }
+{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29" }
+{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18" }
+{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28" }
+{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10" }
+{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03" }
+{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11" }
+{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09" }
+{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27" }
+{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25" }
+{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17" }
+{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06" }
+{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21" }
+{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01" }
+{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12" }
+{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11" }
+{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14" }
+{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22" }
+{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02" }
+{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31" }
+{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15" }
+{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06" }
+{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20" }
+{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07" }
+{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31" }
+{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05" }
+{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07" }
 { "partkey": 170, "pid": 1, "shipdate": "1992-08-07" }
 { "partkey": 170, "pid": 2, "shipdate": "1993-03-17" }
 { "partkey": 170, "pid": 3, "shipdate": "1993-06-19" }
 { "partkey": 171, "pid": 1, "shipdate": "1992-11-09" }
 { "partkey": 171, "pid": 2, "shipdate": "1994-01-22" }
 { "partkey": 171, "pid": 3, "shipdate": "1995-01-02" }
+{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06" }
+{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01" }
+{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16" }
+{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17" }
+{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15" }
+{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30" }
+{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25" }
+{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02" }
+{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02" }
+{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09" }
+{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09" }
+{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10" }
+{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01" }
+{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28" }
+{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24" }
+{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05" }
+{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25" }
+{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16" }
 { "partkey": 178, "pid": 1, "shipdate": "1992-05-23" }
 { "partkey": 178, "pid": 2, "shipdate": "1992-08-18" }
 { "partkey": 178, "pid": 3, "shipdate": "1992-11-02" }
@@ -130,471 +538,63 @@
 { "partkey": 180, "pid": 1, "shipdate": "1992-03-07" }
 { "partkey": 180, "pid": 2, "shipdate": "1992-05-23" }
 { "partkey": 180, "pid": 3, "shipdate": "1992-06-21" }
+{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01" }
+{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04" }
+{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14" }
 { "partkey": 182, "pid": 1, "shipdate": "1992-03-02" }
 { "partkey": 182, "pid": 2, "shipdate": "1992-04-02" }
 { "partkey": 182, "pid": 3, "shipdate": "1992-04-28" }
+{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24" }
+{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24" }
+{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08" }
+{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12" }
+{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12" }
+{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30" }
+{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30" }
+{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23" }
+{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26" }
+{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25" }
+{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27" }
+{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01" }
+{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30" }
+{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01" }
+{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15" }
+{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08" }
+{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03" }
+{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16" }
+{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20" }
 { "partkey": 190, "pid": 1, "shipdate": "1992-04-14" }
 { "partkey": 190, "pid": 2, "shipdate": "1992-07-17" }
 { "partkey": 190, "pid": 3, "shipdate": "1992-10-12" }
+{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31" }
+{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29" }
+{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22" }
+{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19" }
+{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10" }
+{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02" }
+{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05" }
+{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21" }
+{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12" }
+{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14" }
+{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20" }
+{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15" }
 { "partkey": 195, "pid": 1, "shipdate": "1992-04-10" }
 { "partkey": 195, "pid": 2, "shipdate": "1992-05-07" }
 { "partkey": 195, "pid": 3, "shipdate": "1992-05-28" }
+{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02" }
+{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04" }
+{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11" }
+{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22" }
+{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24" }
+{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03" }
+{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21" }
+{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12" }
+{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27" }
 { "partkey": 199, "pid": 1, "shipdate": "1992-03-14" }
 { "partkey": 199, "pid": 2, "shipdate": "1992-08-02" }
 { "partkey": 199, "pid": 3, "shipdate": "1992-11-20" }
-{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15" }
-{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30" }
-{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17" }
-{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23" }
-{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01" }
-{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18" }
-{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02" }
-{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03" }
-{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18" }
-{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01" }
-{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26" }
-{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04" }
-{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18" }
-{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24" }
-{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14" }
-{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11" }
-{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25" }
-{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17" }
-{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19" }
-{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21" }
-{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22" }
-{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15" }
-{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29" }
-{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18" }
-{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21" }
-{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25" }
-{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20" }
-{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06" }
-{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08" }
-{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25" }
-{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01" }
-{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25" }
-{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14" }
-{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24" }
-{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29" }
-{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11" }
-{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06" }
-{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26" }
-{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28" }
-{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08" }
-{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21" }
-{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10" }
-{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03" }
-{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15" }
-{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31" }
-{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03" }
-{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11" }
-{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17" }
-{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02" }
-{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18" }
-{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16" }
-{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06" }
-{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09" }
-{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17" }
-{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12" }
-{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26" }
-{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19" }
-{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07" }
-{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15" }
-{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07" }
-{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13" }
-{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14" }
-{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10" }
-{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02" }
-{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14" }
-{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26" }
-{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13" }
-{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08" }
-{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22" }
-{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10" }
-{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10" }
-{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28" }
-{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08" }
-{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16" }
-{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02" }
-{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27" }
-{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12" }
-{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19" }
-{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11" }
-{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22" }
-{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30" }
-{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22" }
-{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21" }
-{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03" }
-{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27" }
-{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22" }
-{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21" }
-{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06" }
-{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09" }
-{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09" }
-{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24" }
-{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24" }
-{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18" }
-{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09" }
-{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31" }
-{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02" }
-{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19" }
-{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22" }
-{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22" }
-{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08" }
-{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27" }
-{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07" }
-{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23" }
-{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09" }
-{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23" }
-{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12" }
-{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09" }
-{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05" }
-{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04" }
-{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02" }
-{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13" }
-{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31" }
-{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28" }
-{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15" }
-{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03" }
-{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23" }
-{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20" }
-{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27" }
-{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03" }
-{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14" }
-{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20" }
-{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27" }
-{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03" }
-{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17" }
-{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01" }
-{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05" }
-{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25" }
-{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17" }
-{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22" }
-{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29" }
-{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14" }
-{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01" }
-{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29" }
-{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18" }
-{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03" }
-{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10" }
-{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29" }
-{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18" }
-{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28" }
-{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10" }
-{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03" }
-{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11" }
-{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09" }
-{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27" }
-{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25" }
-{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17" }
-{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06" }
-{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21" }
-{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01" }
-{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12" }
-{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06" }
-{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20" }
-{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07" }
-{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06" }
-{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01" }
-{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16" }
-{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09" }
-{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09" }
-{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10" }
-{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05" }
-{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25" }
-{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16" }
-{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01" }
-{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04" }
-{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14" }
-{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12" }
-{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30" }
-{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26" }
-{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27" }
-{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16" }
-{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20" }
-{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14" }
-{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15" }
-{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22" }
-{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24" }
-{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03" }
-{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21" }
-{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12" }
-{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27" }
-{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25" }
-{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15" }
-{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13" }
-{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29" }
-{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30" }
-{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13" }
-{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01" }
-{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04" }
-{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23" }
-{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01" }
-{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16" }
-{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13" }
-{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04" }
-{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03" }
-{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20" }
-{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23" }
-{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30" }
-{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03" }
-{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31" }
-{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07" }
-{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03" }
-{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13" }
-{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18" }
-{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13" }
-{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30" }
-{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28" }
-{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11" }
-{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10" }
-{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13" }
-{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08" }
-{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03" }
-{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31" }
-{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05" }
-{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01" }
-{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22" }
-{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19" }
-{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12" }
-{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05" }
-{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10" }
-{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08" }
-{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08" }
-{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15" }
-{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20" }
-{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28" }
-{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28" }
-{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27" }
-{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25" }
-{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18" }
-{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01" }
-{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20" }
-{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03" }
-{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26" }
-{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24" }
-{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14" }
-{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17" }
-{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18" }
-{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26" }
-{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25" }
-{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01" }
-{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18" }
-{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09" }
-{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19" }
-{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21" }
-{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25" }
-{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28" }
-{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01" }
-{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07" }
-{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28" }
-{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13" }
-{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08" }
-{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13" }
-{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25" }
-{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22" }
-{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17" }
-{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24" }
-{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15" }
-{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09" }
-{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13" }
-{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05" }
-{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02" }
-{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24" }
-{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17" }
-{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14" }
-{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06" }
-{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08" }
-{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17" }
-{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18" }
-{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17" }
-{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20" }
-{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29" }
-{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19" }
-{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21" }
-{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07" }
-{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28" }
-{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12" }
-{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01" }
-{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02" }
-{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25" }
-{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28" }
-{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25" }
-{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14" }
-{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07" }
-{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04" }
-{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18" }
-{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11" }
-{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14" }
-{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22" }
-{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02" }
-{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31" }
-{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15" }
-{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25" }
-{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02" }
-{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02" }
-{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24" }
-{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24" }
-{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08" }
-{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30" }
-{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23" }
-{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01" }
-{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30" }
-{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01" }
-{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31" }
-{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29" }
-{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22" }
-{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19" }
-{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10" }
-{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02" }
-{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02" }
-{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04" }
-{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11" }
 { "partkey": 200, "pid": 1, "shipdate": "1992-04-19" }
 { "partkey": 200, "pid": 2, "shipdate": "1993-01-06" }
 { "partkey": 200, "pid": 3, "shipdate": "1993-10-17" }
-{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25" }
-{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24" }
-{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03" }
-{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02" }
-{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14" }
-{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11" }
-{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25" }
-{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23" }
-{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01" }
-{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06" }
-{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12" }
-{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21" }
-{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21" }
-{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05" }
-{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14" }
-{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17" }
-{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22" }
-{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25" }
-{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07" }
-{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26" }
-{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03" }
-{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26" }
-{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12" }
-{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15" }
-{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23" }
-{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04" }
-{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12" }
-{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14" }
-{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11" }
-{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29" }
-{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22" }
-{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31" }
-{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23" }
-{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14" }
-{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22" }
-{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04" }
-{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07" }
-{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01" }
-{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24" }
-{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14" }
-{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15" }
-{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29" }
-{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21" }
-{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22" }
-{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21" }
-{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04" }
-{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04" }
-{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06" }
-{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18" }
-{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02" }
-{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07" }
-{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17" }
-{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18" }
-{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11" }
-{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09" }
-{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04" }
-{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21" }
-{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30" }
-{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02" }
-{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06" }
-{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25" }
-{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07" }
-{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21" }
-{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17" }
-{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27" }
-{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28" }
-{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17" }
-{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08" }
-{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22" }
-{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22" }
-{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30" }
-{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05" }
-{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18" }
-{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01" }
-{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01" }
-{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13" }
-{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09" }
-{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15" }
-{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04" }
-{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18" }
-{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10" }
-{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23" }
-{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29" }
-{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20" }
-{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22" }
-{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23" }
-{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05" }
-{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12" }
-{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14" }
-{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14" }
-{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11" }
-{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25" }
-{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16" }
-{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25" }
-{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21" }
-{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21" }
-{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02" }
-{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15" }
-{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27" }
-{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22" }
-{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26" }
-{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30" }
-{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19" }
-{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31" }
-{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05" }
-{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07" }
-{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17" }
-{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15" }
-{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30" }
-{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01" }
-{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28" }
-{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24" }
-{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15" }
-{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08" }
-{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03" }
-{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05" }
-{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21" }
-{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12" }


[3/8] asterixdb git commit: Supports flexible degree of parallelism.

Posted by bu...@apache.org.
http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at06/at06.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at06/at06.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at06/at06.1.adm
index 11338af..53d995c 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at06/at06.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/flwor/at06/at06.1.adm
@@ -1,105 +1,456 @@
+{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15", "orderkey": 5409 }
+{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30", "orderkey": 1154 }
+{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17", "orderkey": 134 }
+{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23", "orderkey": 3650 }
+{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01", "orderkey": 130 }
+{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18", "orderkey": 5893 }
+{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25", "orderkey": 801 }
+{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24", "orderkey": 194 }
+{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03", "orderkey": 3776 }
+{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02", "orderkey": 4292 }
+{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03", "orderkey": 164 }
+{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18", "orderkey": 2019 }
+{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02", "orderkey": 3970 }
+{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14", "orderkey": 5959 }
+{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06", "orderkey": 3680 }
 { "partkey": 6, "pid": 1, "shipdate": "1992-04-05", "orderkey": 4483 }
 { "partkey": 6, "pid": 2, "shipdate": "1992-04-25", "orderkey": 801 }
 { "partkey": 6, "pid": 3, "shipdate": "1992-04-29", "orderkey": 2689 }
+{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12", "orderkey": 3140 }
+{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11", "orderkey": 3204 }
+{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25", "orderkey": 5794 }
+{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25", "orderkey": 5635 }
+{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15", "orderkey": 1540 }
+{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13", "orderkey": 1222 }
+{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29", "orderkey": 3970 }
+{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30", "orderkey": 1955 }
+{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01", "orderkey": 4199 }
+{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13", "orderkey": 2881 }
+{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25", "orderkey": 5378 }
+{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01", "orderkey": 1796 }
 { "partkey": 11, "pid": 1, "shipdate": "1992-02-14", "orderkey": 4800 }
 { "partkey": 11, "pid": 2, "shipdate": "1992-07-20", "orderkey": 5858 }
 { "partkey": 11, "pid": 3, "shipdate": "1992-08-03", "orderkey": 3237 }
 { "partkey": 12, "pid": 1, "shipdate": "1992-07-04", "orderkey": 130 }
 { "partkey": 12, "pid": 2, "shipdate": "1992-07-17", "orderkey": 322 }
 { "partkey": 12, "pid": 3, "shipdate": "1992-09-02", "orderkey": 2497 }
+{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01", "orderkey": 1537 }
+{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26", "orderkey": 322 }
+{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04", "orderkey": 5953 }
 { "partkey": 14, "pid": 1, "shipdate": "1992-07-17", "orderkey": 5028 }
 { "partkey": 14, "pid": 2, "shipdate": "1992-11-30", "orderkey": 3232 }
 { "partkey": 14, "pid": 3, "shipdate": "1993-05-10", "orderkey": 2279 }
+{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18", "orderkey": 5473 }
+{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24", "orderkey": 2688 }
+{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14", "orderkey": 5472 }
+{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11", "orderkey": 1346 }
+{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25", "orderkey": 5858 }
+{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17", "orderkey": 5415 }
+{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23", "orderkey": 967 }
+{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01", "orderkey": 931 }
+{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06", "orderkey": 611 }
+{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12", "orderkey": 1537 }
+{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21", "orderkey": 2880 }
+{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21", "orderkey": 2688 }
+{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19", "orderkey": 2023 }
+{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21", "orderkey": 481 }
+{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22", "orderkey": 164 }
+{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15", "orderkey": 2023 }
+{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29", "orderkey": 5254 }
+{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18", "orderkey": 2625 }
 { "partkey": 21, "pid": 1, "shipdate": "1992-07-31", "orderkey": 549 }
 { "partkey": 21, "pid": 2, "shipdate": "1992-09-09", "orderkey": 4581 }
 { "partkey": 21, "pid": 3, "shipdate": "1993-01-09", "orderkey": 481 }
+{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21", "orderkey": 1285 }
+{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25", "orderkey": 3970 }
+{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20", "orderkey": 1447 }
 { "partkey": 23, "pid": 1, "shipdate": "1992-04-04", "orderkey": 2786 }
 { "partkey": 23, "pid": 2, "shipdate": "1992-06-19", "orderkey": 1856 }
 { "partkey": 23, "pid": 3, "shipdate": "1992-06-29", "orderkey": 1282 }
+{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12", "orderkey": 2755 }
+{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06", "orderkey": 4260 }
+{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08", "orderkey": 3845 }
+{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04", "orderkey": 2688 }
+{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23", "orderkey": 5060 }
+{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01", "orderkey": 868 }
 { "partkey": 26, "pid": 1, "shipdate": "1992-02-23", "orderkey": 4800 }
 { "partkey": 26, "pid": 2, "shipdate": "1992-05-09", "orderkey": 801 }
 { "partkey": 26, "pid": 3, "shipdate": "1993-01-04", "orderkey": 2146 }
+{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05", "orderkey": 1826 }
+{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14", "orderkey": 4096 }
+{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17", "orderkey": 4294 }
+{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16", "orderkey": 2240 }
+{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13", "orderkey": 5699 }
+{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04", "orderkey": 1506 }
+{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25", "orderkey": 4738 }
+{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01", "orderkey": 3205 }
+{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25", "orderkey": 868 }
 { "partkey": 30, "pid": 1, "shipdate": "1992-04-10", "orderkey": 1282 }
 { "partkey": 30, "pid": 2, "shipdate": "1992-05-18", "orderkey": 1925 }
 { "partkey": 30, "pid": 3, "shipdate": "1992-05-21", "orderkey": 5986 }
+{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14", "orderkey": 4705 }
+{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24", "orderkey": 1185 }
+{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29", "orderkey": 5415 }
+{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22", "orderkey": 4900 }
+{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25", "orderkey": 5060 }
+{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07", "orderkey": 5603 }
 { "partkey": 33, "pid": 1, "shipdate": "1992-03-22", "orderkey": 5574 }
 { "partkey": 33, "pid": 2, "shipdate": "1993-02-17", "orderkey": 4163 }
 { "partkey": 33, "pid": 3, "shipdate": "1993-02-21", "orderkey": 388 }
+{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03", "orderkey": 322 }
+{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20", "orderkey": 3845 }
+{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23", "orderkey": 5089 }
+{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11", "orderkey": 4230 }
+{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06", "orderkey": 4804 }
+{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26", "orderkey": 2880 }
+{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26", "orderkey": 1154 }
+{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03", "orderkey": 134 }
+{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06", "orderkey": 3521 }
+{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30", "orderkey": 1088 }
+{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03", "orderkey": 2500 }
+{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31", "orderkey": 3074 }
 { "partkey": 38, "pid": 1, "shipdate": "1992-04-06", "orderkey": 5601 }
 { "partkey": 38, "pid": 2, "shipdate": "1992-04-15", "orderkey": 322 }
 { "partkey": 38, "pid": 3, "shipdate": "1992-08-27", "orderkey": 2023 }
+{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26", "orderkey": 4515 }
+{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12", "orderkey": 612 }
+{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15", "orderkey": 1447 }
+{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07", "orderkey": 4292 }
+{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28", "orderkey": 3139 }
+{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03", "orderkey": 3973 }
+{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13", "orderkey": 4896 }
+{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18", "orderkey": 2852 }
+{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13", "orderkey": 3367 }
+{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23", "orderkey": 2560 }
+{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04", "orderkey": 2566 }
+{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12", "orderkey": 1571 }
+{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18", "orderkey": 4069 }
+{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30", "orderkey": 2052 }
+{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28", "orderkey": 5959 }
+{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14", "orderkey": 4292 }
+{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11", "orderkey": 322 }
+{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29", "orderkey": 2147 }
 { "partkey": 45, "pid": 1, "shipdate": "1992-07-16", "orderkey": 4515 }
 { "partkey": 45, "pid": 2, "shipdate": "1993-06-24", "orderkey": 2720 }
 { "partkey": 45, "pid": 3, "shipdate": "1993-09-15", "orderkey": 2055 }
+{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28", "orderkey": 4230 }
+{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08", "orderkey": 3043 }
+{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21", "orderkey": 3845 }
 { "partkey": 47, "pid": 1, "shipdate": "1992-03-11", "orderkey": 3685 }
 { "partkey": 47, "pid": 2, "shipdate": "1993-05-30", "orderkey": 3171 }
 { "partkey": 47, "pid": 3, "shipdate": "1993-06-06", "orderkey": 2341 }
+{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10", "orderkey": 2691 }
+{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03", "orderkey": 5473 }
+{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15", "orderkey": 832 }
 { "partkey": 49, "pid": 1, "shipdate": "1992-04-29", "orderkey": 2983 }
 { "partkey": 49, "pid": 2, "shipdate": "1992-06-14", "orderkey": 2022 }
 { "partkey": 49, "pid": 3, "shipdate": "1992-08-13", "orderkey": 933 }
+{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22", "orderkey": 2786 }
+{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31", "orderkey": 644 }
+{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23", "orderkey": 2885 }
 { "partkey": 51, "pid": 1, "shipdate": "1992-03-11", "orderkey": 5860 }
 { "partkey": 51, "pid": 2, "shipdate": "1992-05-15", "orderkey": 2786 }
 { "partkey": 51, "pid": 3, "shipdate": "1992-05-17", "orderkey": 644 }
+{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31", "orderkey": 1057 }
+{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03", "orderkey": 4838 }
+{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21", "orderkey": 3907 }
+{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14", "orderkey": 4800 }
+{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22", "orderkey": 2240 }
+{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04", "orderkey": 2562 }
+{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07", "orderkey": 4515 }
+{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01", "orderkey": 3271 }
+{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24", "orderkey": 1701 }
+{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16", "orderkey": 5382 }
+{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11", "orderkey": 1856 }
+{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17", "orderkey": 2022 }
+{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16", "orderkey": 1248 }
+{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02", "orderkey": 3685 }
+{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18", "orderkey": 3205 }
+{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16", "orderkey": 3271 }
+{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06", "orderkey": 194 }
+{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21", "orderkey": 2146 }
 { "partkey": 58, "pid": 1, "shipdate": "1992-05-16", "orderkey": 3685 }
 { "partkey": 58, "pid": 2, "shipdate": "1992-10-30", "orderkey": 4896 }
 { "partkey": 58, "pid": 3, "shipdate": "1993-04-10", "orderkey": 1412 }
+{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09", "orderkey": 2688 }
+{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17", "orderkey": 4998 }
+{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12", "orderkey": 3845 }
 { "partkey": 60, "pid": 1, "shipdate": "1992-02-14", "orderkey": 3168 }
 { "partkey": 60, "pid": 2, "shipdate": "1992-07-01", "orderkey": 1217 }
 { "partkey": 60, "pid": 3, "shipdate": "1992-07-15", "orderkey": 3043 }
+{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14", "orderkey": 2020 }
+{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15", "orderkey": 5318 }
+{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29", "orderkey": 261 }
+{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01", "orderkey": 1248 }
+{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26", "orderkey": 5382 }
+{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19", "orderkey": 4483 }
+{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07", "orderkey": 4998 }
+{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15", "orderkey": 3650 }
+{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07", "orderkey": 4545 }
+{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13", "orderkey": 2755 }
+{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14", "orderkey": 5409 }
+{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10", "orderkey": 3271 }
+{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02", "orderkey": 4804 }
+{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14", "orderkey": 2848 }
+{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26", "orderkey": 5095 }
+{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07", "orderkey": 194 }
+{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11", "orderkey": 549 }
+{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10", "orderkey": 3015 }
+{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13", "orderkey": 1764 }
+{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08", "orderkey": 612 }
+{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03", "orderkey": 2631 }
+{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13", "orderkey": 3842 }
+{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08", "orderkey": 5121 }
+{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22", "orderkey": 868 }
+{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31", "orderkey": 3205 }
+{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05", "orderkey": 5767 }
+{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01", "orderkey": 1221 }
 { "partkey": 70, "pid": 1, "shipdate": "1992-04-06", "orderkey": 5473 }
 { "partkey": 70, "pid": 2, "shipdate": "1992-06-11", "orderkey": 4199 }
 { "partkey": 70, "pid": 3, "shipdate": "1992-06-25", "orderkey": 3650 }
+{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10", "orderkey": 2497 }
+{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10", "orderkey": 2146 }
+{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28", "orderkey": 4611 }
 { "partkey": 72, "pid": 1, "shipdate": "1992-09-16", "orderkey": 3265 }
 { "partkey": 72, "pid": 2, "shipdate": "1992-10-02", "orderkey": 5635 }
 { "partkey": 72, "pid": 3, "shipdate": "1992-10-17", "orderkey": 3655 }
+{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08", "orderkey": 5601 }
+{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16", "orderkey": 4741 }
+{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02", "orderkey": 4743 }
+{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21", "orderkey": 4162 }
+{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22", "orderkey": 801 }
+{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21", "orderkey": 929 }
+{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27", "orderkey": 4000 }
+{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12", "orderkey": 4230 }
+{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19", "orderkey": 4103 }
+{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22", "orderkey": 5408 }
+{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19", "orderkey": 2272 }
+{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12", "orderkey": 2245 }
 { "partkey": 77, "pid": 1, "shipdate": "1992-08-18", "orderkey": 4900 }
 { "partkey": 77, "pid": 2, "shipdate": "1992-12-23", "orderkey": 2497 }
 { "partkey": 77, "pid": 3, "shipdate": "1993-06-19", "orderkey": 4166 }
+{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04", "orderkey": 2210 }
+{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04", "orderkey": 2022 }
+{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06", "orderkey": 1764 }
+{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05", "orderkey": 4069 }
+{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10", "orderkey": 5986 }
+{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08", "orderkey": 4418 }
+{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18", "orderkey": 644 }
+{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02", "orderkey": 2500 }
+{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07", "orderkey": 3877 }
+{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11", "orderkey": 2240 }
+{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22", "orderkey": 1221 }
+{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30", "orderkey": 5954 }
+{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17", "orderkey": 4867 }
+{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18", "orderkey": 1504 }
+{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11", "orderkey": 4261 }
+{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09", "orderkey": 4738 }
+{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04", "orderkey": 5218 }
+{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21", "orderkey": 5220 }
+{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08", "orderkey": 1285 }
+{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15", "orderkey": 2597 }
+{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20", "orderkey": 772 }
+{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28", "orderkey": 1057 }
+{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28", "orderkey": 5574 }
+{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27", "orderkey": 1221 }
+{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25", "orderkey": 2240 }
+{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18", "orderkey": 4896 }
+{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01", "orderkey": 4166 }
+{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30", "orderkey": 4294 }
+{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02", "orderkey": 1540 }
+{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06", "orderkey": 3556 }
 { "partkey": 88, "pid": 1, "shipdate": "1992-04-24", "orderkey": 3970 }
 { "partkey": 88, "pid": 2, "shipdate": "1992-06-26", "orderkey": 3842 }
 { "partkey": 88, "pid": 3, "shipdate": "1992-12-18", "orderkey": 612 }
 { "partkey": 89, "pid": 1, "shipdate": "1992-04-18", "orderkey": 2688 }
 { "partkey": 89, "pid": 2, "shipdate": "1992-04-19", "orderkey": 4705 }
 { "partkey": 89, "pid": 3, "shipdate": "1992-05-27", "orderkey": 5121 }
+{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25", "orderkey": 4162 }
+{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07", "orderkey": 5474 }
+{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21", "orderkey": 5986 }
+{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22", "orderkey": 3043 }
+{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21", "orderkey": 2691 }
+{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03", "orderkey": 3015 }
 { "partkey": 92, "pid": 1, "shipdate": "1992-02-11", "orderkey": 2755 }
 { "partkey": 92, "pid": 2, "shipdate": "1992-09-30", "orderkey": 487 }
 { "partkey": 92, "pid": 3, "shipdate": "1993-01-04", "orderkey": 164 }
 { "partkey": 93, "pid": 1, "shipdate": "1992-05-28", "orderkey": 2881 }
 { "partkey": 93, "pid": 2, "shipdate": "1992-06-24", "orderkey": 384 }
 { "partkey": 93, "pid": 3, "shipdate": "1992-09-11", "orderkey": 3654 }
+{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20", "orderkey": 5574 }
+{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03", "orderkey": 3650 }
+{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26", "orderkey": 3654 }
+{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24", "orderkey": 3271 }
+{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14", "orderkey": 801 }
+{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17", "orderkey": 2176 }
+{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18", "orderkey": 2052 }
+{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26", "orderkey": 3172 }
+{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25", "orderkey": 1159 }
+{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27", "orderkey": 4800 }
+{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22", "orderkey": 1856 }
+{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21", "orderkey": 4035 }
+{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06", "orderkey": 5603 }
+{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09", "orderkey": 1159 }
+{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09", "orderkey": 678 }
+{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01", "orderkey": 4998 }
+{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18", "orderkey": 1409 }
+{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09", "orderkey": 2149 }
+{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24", "orderkey": 292 }
+{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24", "orderkey": 2022 }
+{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18", "orderkey": 4738 }
+{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17", "orderkey": 644 }
+{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27", "orderkey": 2147 }
+{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28", "orderkey": 1571 }
+{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19", "orderkey": 5415 }
+{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21", "orderkey": 5408 }
+{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25", "orderkey": 674 }
 { "partkey": 103, "pid": 1, "shipdate": "1992-03-28", "orderkey": 4515 }
 { "partkey": 103, "pid": 2, "shipdate": "1992-05-08", "orderkey": 832 }
 { "partkey": 103, "pid": 3, "shipdate": "1992-07-11", "orderkey": 4900 }
+{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17", "orderkey": 5409 }
+{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08", "orderkey": 4897 }
+{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22", "orderkey": 5479 }
 { "partkey": 105, "pid": 1, "shipdate": "1992-02-14", "orderkey": 5382 }
 { "partkey": 105, "pid": 2, "shipdate": "1992-06-01", "orderkey": 615 }
 { "partkey": 105, "pid": 3, "shipdate": "1992-07-14", "orderkey": 4900 }
+{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09", "orderkey": 5095 }
+{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31", "orderkey": 3681 }
+{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02", "orderkey": 967 }
+{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22", "orderkey": 1088 }
+{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30", "orderkey": 3654 }
+{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05", "orderkey": 3842 }
+{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28", "orderkey": 1826 }
+{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01", "orderkey": 1221 }
+{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07", "orderkey": 2560 }
 { "partkey": 109, "pid": 1, "shipdate": "1992-06-06", "orderkey": 3970 }
 { "partkey": 109, "pid": 2, "shipdate": "1992-11-20", "orderkey": 1159 }
 { "partkey": 109, "pid": 3, "shipdate": "1992-12-23", "orderkey": 164 }
-{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13", "orderkey": 2912 }
-{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29", "orderkey": 3973 }
-{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17", "orderkey": 2880 }
-{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18", "orderkey": 4035 }
-{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27", "orderkey": 1793 }
-{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02", "orderkey": 5408 }
-{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15", "orderkey": 2848 }
-{ "partkey": 125, "pid": 2, "shipdate": "1992-03-29", "orderkey": 4230 }
+{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18", "orderkey": 3907 }
+{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01", "orderkey": 4261 }
+{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01", "orderkey": 1991 }
+{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05", "orderkey": 4705 }
+{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28", "orderkey": 5254 }
+{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13", "orderkey": 5121 }
+{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13", "orderkey": 3907 }
+{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09", "orderkey": 2885 }
+{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15", "orderkey": 481 }
+{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08", "orderkey": 1027 }
+{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13", "orderkey": 2054 }
+{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25", "orderkey": 4741 }
+{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19", "orderkey": 3014 }
+{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22", "orderkey": 1506 }
+{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22", "orderkey": 710 }
+{ "partkey": 115, "pid": 1, "shipdate": "1992-03-13", "orderkey": 2912 }
+{ "partkey": 115, "pid": 2, "shipdate": "1992-05-29", "orderkey": 3973 }
+{ "partkey": 115, "pid": 3, "shipdate": "1992-06-17", "orderkey": 2880 }
+{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22", "orderkey": 2755 }
+{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17", "orderkey": 1925 }
+{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24", "orderkey": 5603 }
+{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04", "orderkey": 1856 }
+{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18", "orderkey": 4545 }
+{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10", "orderkey": 3494 }
+{ "partkey": 118, "pid": 1, "shipdate": "1992-06-18", "orderkey": 4035 }
+{ "partkey": 118, "pid": 2, "shipdate": "1992-09-27", "orderkey": 1793 }
+{ "partkey": 118, "pid": 3, "shipdate": "1992-10-02", "orderkey": 5408 }
+{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08", "orderkey": 5574 }
+{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27", "orderkey": 5959 }
+{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07", "orderkey": 4294 }
+{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23", "orderkey": 4292 }
+{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28", "orderkey": 1221 }
+{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29", "orderkey": 4903 }
+{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23", "orderkey": 4903 }
+{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09", "orderkey": 1764 }
+{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23", "orderkey": 2054 }
+{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12", "orderkey": 1248 }
+{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09", "orderkey": 2912 }
+{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05", "orderkey": 801 }
+{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01", "orderkey": 3011 }
+{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20", "orderkey": 5095 }
+{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22", "orderkey": 1505 }
+{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15", "orderkey": 1088 }
+{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09", "orderkey": 2209 }
+{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13", "orderkey": 1346 }
+{ "partkey": 125, "pid": 1, "shipdate": "1992-03-15", "orderkey": 2848 }
+{ "partkey": 125, "pid": 2, "shipdate": "1992-03-29", "orderkey": 4230 }
 { "partkey": 125, "pid": 3, "shipdate": "1992-05-24", "orderkey": 4069 }
 { "partkey": 126, "pid": 1, "shipdate": "1992-07-28", "orderkey": 1793 }
 { "partkey": 126, "pid": 2, "shipdate": "1992-08-28", "orderkey": 1027 }
 { "partkey": 126, "pid": 3, "shipdate": "1992-09-06", "orderkey": 3907 }
+{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04", "orderkey": 2023 }
+{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02", "orderkey": 37 }
+{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13", "orderkey": 1316 }
+{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05", "orderkey": 3168 }
+{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02", "orderkey": 4804 }
+{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24", "orderkey": 4096 }
+{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31", "orderkey": 2022 }
+{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28", "orderkey": 5953 }
+{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15", "orderkey": 130 }
+{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03", "orderkey": 4705 }
+{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23", "orderkey": 1856 }
+{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20", "orderkey": 644 }
+{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27", "orderkey": 2755 }
+{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03", "orderkey": 4292 }
+{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14", "orderkey": 2627 }
+{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17", "orderkey": 5607 }
+{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14", "orderkey": 384 }
+{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06", "orderkey": 3172 }
+{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08", "orderkey": 3140 }
+{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17", "orderkey": 4864 }
+{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18", "orderkey": 1506 }
+{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17", "orderkey": 3685 }
+{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20", "orderkey": 644 }
+{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29", "orderkey": 421 }
 { "partkey": 135, "pid": 1, "shipdate": "1992-05-02", "orderkey": 1826 }
 { "partkey": 135, "pid": 2, "shipdate": "1992-05-11", "orderkey": 1925 }
 { "partkey": 135, "pid": 3, "shipdate": "1992-05-29", "orderkey": 2052 }
+{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19", "orderkey": 2786 }
+{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21", "orderkey": 4035 }
+{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07", "orderkey": 4805 }
+{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23", "orderkey": 3524 }
+{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05", "orderkey": 1955 }
+{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12", "orderkey": 4099 }
 { "partkey": 138, "pid": 1, "shipdate": "1992-06-20", "orderkey": 2848 }
 { "partkey": 138, "pid": 2, "shipdate": "1992-11-21", "orderkey": 1991 }
 { "partkey": 138, "pid": 3, "shipdate": "1993-02-28", "orderkey": 4487 }
+{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12", "orderkey": 2880 }
+{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28", "orderkey": 4992 }
+{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12", "orderkey": 4099 }
+{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20", "orderkey": 1537 }
+{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27", "orderkey": 6 }
+{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03", "orderkey": 2881 }
 { "partkey": 141, "pid": 1, "shipdate": "1992-01-13", "orderkey": 5409 }
 { "partkey": 141, "pid": 2, "shipdate": "1992-02-01", "orderkey": 3712 }
 { "partkey": 141, "pid": 3, "shipdate": "1992-06-22", "orderkey": 1344 }
+{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14", "orderkey": 3556 }
+{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14", "orderkey": 2241 }
+{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11", "orderkey": 5670 }
+{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17", "orderkey": 1154 }
+{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01", "orderkey": 3524 }
+{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05", "orderkey": 1285 }
+{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05", "orderkey": 4992 }
+{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25", "orderkey": 5415 }
+{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17", "orderkey": 4996 }
+{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25", "orderkey": 4998 }
+{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16", "orderkey": 134 }
+{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25", "orderkey": 3907 }
+{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21", "orderkey": 194 }
+{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21", "orderkey": 678 }
+{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02", "orderkey": 1286 }
 { "partkey": 147, "pid": 1, "shipdate": "1992-06-10", "orderkey": 5959 }
 { "partkey": 147, "pid": 2, "shipdate": "1992-09-04", "orderkey": 4992 }
 { "partkey": 147, "pid": 3, "shipdate": "1992-12-03", "orderkey": 614 }
+{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15", "orderkey": 3712 }
+{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27", "orderkey": 5601 }
+{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22", "orderkey": 1154 }
+{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22", "orderkey": 5382 }
+{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29", "orderkey": 2688 }
+{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14", "orderkey": 194 }
+{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01", "orderkey": 4805 }
+{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02", "orderkey": 1856 }
+{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25", "orderkey": 1701 }
+{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26", "orderkey": 1248 }
+{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30", "orderkey": 4256 }
+{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19", "orderkey": 3014 }
 { "partkey": 152, "pid": 1, "shipdate": "1992-06-23", "orderkey": 4230 }
 { "partkey": 152, "pid": 2, "shipdate": "1993-05-19", "orderkey": 896 }
 { "partkey": 152, "pid": 3, "shipdate": "1993-10-31", "orderkey": 2368 }
@@ -109,18 +460,75 @@
 { "partkey": 154, "pid": 1, "shipdate": "1992-02-18", "orderkey": 292 }
 { "partkey": 154, "pid": 2, "shipdate": "1992-02-20", "orderkey": 4998 }
 { "partkey": 154, "pid": 3, "shipdate": "1992-05-14", "orderkey": 4805 }
+{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28", "orderkey": 1956 }
+{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25", "orderkey": 5378 }
+{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14", "orderkey": 2305 }
 { "partkey": 156, "pid": 1, "shipdate": "1992-04-24", "orderkey": 1248 }
 { "partkey": 156, "pid": 2, "shipdate": "1992-06-17", "orderkey": 1027 }
 { "partkey": 156, "pid": 3, "shipdate": "1992-07-01", "orderkey": 2786 }
 { "partkey": 157, "pid": 1, "shipdate": "1992-07-26", "orderkey": 4069 }
 { "partkey": 157, "pid": 2, "shipdate": "1992-08-11", "orderkey": 1729 }
 { "partkey": 157, "pid": 3, "shipdate": "1992-08-25", "orderkey": 4741 }
+{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01", "orderkey": 1955 }
+{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29", "orderkey": 5254 }
+{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18", "orderkey": 5089 }
+{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07", "orderkey": 5409 }
+{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03", "orderkey": 1955 }
+{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10", "orderkey": 4738 }
+{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07", "orderkey": 1282 }
+{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04", "orderkey": 4867 }
+{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18", "orderkey": 1346 }
+{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29", "orderkey": 2240 }
+{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18", "orderkey": 4391 }
+{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28", "orderkey": 5060 }
+{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10", "orderkey": 5953 }
+{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03", "orderkey": 2786 }
+{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11", "orderkey": 2691 }
+{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09", "orderkey": 2983 }
+{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27", "orderkey": 4292 }
+{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01", "orderkey": 4992 }
+{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25", "orderkey": 5601 }
+{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17", "orderkey": 1248 }
+{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06", "orderkey": 801 }
+{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21", "orderkey": 2848 }
+{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01", "orderkey": 4903 }
+{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12", "orderkey": 3168 }
+{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11", "orderkey": 2691 }
+{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14", "orderkey": 5095 }
+{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22", "orderkey": 1703 }
+{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02", "orderkey": 5767 }
+{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31", "orderkey": 1447 }
+{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15", "orderkey": 1857 }
+{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06", "orderkey": 194 }
+{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20", "orderkey": 3654 }
+{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07", "orderkey": 868 }
+{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31", "orderkey": 1057 }
+{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05", "orderkey": 5953 }
+{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07", "orderkey": 1894 }
 { "partkey": 170, "pid": 1, "shipdate": "1992-08-07", "orderkey": 1221 }
 { "partkey": 170, "pid": 2, "shipdate": "1993-03-17", "orderkey": 738 }
 { "partkey": 170, "pid": 3, "shipdate": "1993-06-19", "orderkey": 3874 }
 { "partkey": 171, "pid": 1, "shipdate": "1992-11-09", "orderkey": 3361 }
 { "partkey": 171, "pid": 2, "shipdate": "1994-01-22", "orderkey": 4675 }
 { "partkey": 171, "pid": 3, "shipdate": "1995-01-02", "orderkey": 5317 }
+{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06", "orderkey": 2247 }
+{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01", "orderkey": 167 }
+{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16", "orderkey": 1600 }
+{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17", "orderkey": 4738 }
+{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15", "orderkey": 3654 }
+{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30", "orderkey": 1540 }
+{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25", "orderkey": 2054 }
+{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02", "orderkey": 1991 }
+{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02", "orderkey": 4261 }
+{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09", "orderkey": 929 }
+{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09", "orderkey": 4294 }
+{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10", "orderkey": 2497 }
+{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01", "orderkey": 4800 }
+{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28", "orderkey": 1826 }
+{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24", "orderkey": 3907 }
+{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05", "orderkey": 5382 }
+{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25", "orderkey": 1956 }
+{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16", "orderkey": 3680 }
 { "partkey": 178, "pid": 1, "shipdate": "1992-05-23", "orderkey": 5095 }
 { "partkey": 178, "pid": 2, "shipdate": "1992-08-18", "orderkey": 2209 }
 { "partkey": 178, "pid": 3, "shipdate": "1992-11-02", "orderkey": 1504 }
@@ -130,471 +538,63 @@
 { "partkey": 180, "pid": 1, "shipdate": "1992-03-07", "orderkey": 5382 }
 { "partkey": 180, "pid": 2, "shipdate": "1992-05-23", "orderkey": 4515 }
 { "partkey": 180, "pid": 3, "shipdate": "1992-06-21", "orderkey": 2881 }
+{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01", "orderkey": 1088 }
+{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04", "orderkey": 2209 }
+{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14", "orderkey": 3232 }
 { "partkey": 182, "pid": 1, "shipdate": "1992-03-02", "orderkey": 1057 }
 { "partkey": 182, "pid": 2, "shipdate": "1992-04-02", "orderkey": 384 }
 { "partkey": 182, "pid": 3, "shipdate": "1992-04-28", "orderkey": 737 }
+{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24", "orderkey": 4998 }
+{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24", "orderkey": 5408 }
+{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08", "orderkey": 1571 }
+{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12", "orderkey": 322 }
+{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12", "orderkey": 1925 }
+{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30", "orderkey": 194 }
+{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30", "orderkey": 3712 }
+{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20", "orderkey": 5574 }
+{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23", "orderkey": 2023 }
+{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26", "orderkey": 4069 }
+{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25", "orderkey": 129 }
+{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27", "orderkey": 481 }
+{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01", "orderkey": 4391 }
+{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30", "orderkey": 4738 }
+{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01", "orderkey": 4738 }
+{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15", "orderkey": 1285 }
+{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08", "orderkey": 5381 }
+{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03", "orderkey": 4226 }
+{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16", "orderkey": 4805 }
+{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20", "orderkey": 134 }
+{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20", "orderkey": 1285 }
 { "partkey": 190, "pid": 1, "shipdate": "1992-04-14", "orderkey": 1856 }
 { "partkey": 190, "pid": 2, "shipdate": "1992-07-17", "orderkey": 1344 }
 { "partkey": 190, "pid": 3, "shipdate": "1992-10-12", "orderkey": 1185 }
+{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31", "orderkey": 5767 }
+{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29", "orderkey": 3361 }
+{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22", "orderkey": 1506 }
+{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19", "orderkey": 3685 }
+{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10", "orderkey": 5254 }
+{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02", "orderkey": 2500 }
+{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05", "orderkey": 1057 }
+{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21", "orderkey": 5795 }
+{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12", "orderkey": 2244 }
+{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14", "orderkey": 5409 }
+{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20", "orderkey": 3842 }
+{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15", "orderkey": 5062 }
 { "partkey": 195, "pid": 1, "shipdate": "1992-04-10", "orderkey": 2848 }
 { "partkey": 195, "pid": 2, "shipdate": "1992-05-07", "orderkey": 3744 }
 { "partkey": 195, "pid": 3, "shipdate": "1992-05-28", "orderkey": 3205 }
+{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02", "orderkey": 4000 }
+{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04", "orderkey": 1154 }
+{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11", "orderkey": 4230 }
+{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22", "orderkey": 2080 }
+{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24", "orderkey": 3138 }
+{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03", "orderkey": 70 }
+{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21", "orderkey": 3011 }
+{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12", "orderkey": 4294 }
+{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27", "orderkey": 1345 }
 { "partkey": 199, "pid": 1, "shipdate": "1992-03-14", "orderkey": 4230 }
 { "partkey": 199, "pid": 2, "shipdate": "1992-08-02", "orderkey": 5028 }
 { "partkey": 199, "pid": 3, "shipdate": "1992-11-20", "orderkey": 3333 }
-{ "partkey": 1, "pid": 1, "shipdate": "1992-02-15", "orderkey": 5409 }
-{ "partkey": 1, "pid": 2, "shipdate": "1992-03-30", "orderkey": 1154 }
-{ "partkey": 1, "pid": 3, "shipdate": "1992-07-17", "orderkey": 134 }
-{ "partkey": 2, "pid": 1, "shipdate": "1992-06-23", "orderkey": 3650 }
-{ "partkey": 2, "pid": 2, "shipdate": "1992-07-01", "orderkey": 130 }
-{ "partkey": 2, "pid": 3, "shipdate": "1992-07-18", "orderkey": 5893 }
-{ "partkey": 4, "pid": 1, "shipdate": "1992-05-02", "orderkey": 4292 }
-{ "partkey": 4, "pid": 2, "shipdate": "1992-11-03", "orderkey": 164 }
-{ "partkey": 4, "pid": 3, "shipdate": "1992-11-18", "orderkey": 2019 }
-{ "partkey": 13, "pid": 1, "shipdate": "1992-04-01", "orderkey": 1537 }
-{ "partkey": 13, "pid": 2, "shipdate": "1992-04-26", "orderkey": 322 }
-{ "partkey": 13, "pid": 3, "shipdate": "1992-05-04", "orderkey": 5953 }
-{ "partkey": 15, "pid": 1, "shipdate": "1992-05-18", "orderkey": 5473 }
-{ "partkey": 15, "pid": 2, "shipdate": "1992-05-24", "orderkey": 2688 }
-{ "partkey": 15, "pid": 3, "shipdate": "1993-04-14", "orderkey": 5472 }
-{ "partkey": 16, "pid": 1, "shipdate": "1992-09-11", "orderkey": 1346 }
-{ "partkey": 16, "pid": 2, "shipdate": "1992-09-25", "orderkey": 5858 }
-{ "partkey": 16, "pid": 3, "shipdate": "1992-11-17", "orderkey": 5415 }
-{ "partkey": 19, "pid": 1, "shipdate": "1992-07-19", "orderkey": 2023 }
-{ "partkey": 19, "pid": 2, "shipdate": "1992-10-21", "orderkey": 481 }
-{ "partkey": 19, "pid": 3, "shipdate": "1992-12-22", "orderkey": 164 }
-{ "partkey": 20, "pid": 1, "shipdate": "1992-06-15", "orderkey": 2023 }
-{ "partkey": 20, "pid": 2, "shipdate": "1992-07-29", "orderkey": 5254 }
-{ "partkey": 20, "pid": 3, "shipdate": "1992-10-18", "orderkey": 2625 }
-{ "partkey": 22, "pid": 1, "shipdate": "1992-06-21", "orderkey": 1285 }
-{ "partkey": 22, "pid": 2, "shipdate": "1992-06-25", "orderkey": 3970 }
-{ "partkey": 22, "pid": 3, "shipdate": "1992-11-20", "orderkey": 1447 }
-{ "partkey": 24, "pid": 1, "shipdate": "1992-04-12", "orderkey": 2755 }
-{ "partkey": 24, "pid": 2, "shipdate": "1992-08-06", "orderkey": 4260 }
-{ "partkey": 24, "pid": 3, "shipdate": "1992-08-08", "orderkey": 3845 }
-{ "partkey": 29, "pid": 1, "shipdate": "1992-05-25", "orderkey": 4738 }
-{ "partkey": 29, "pid": 2, "shipdate": "1992-06-01", "orderkey": 3205 }
-{ "partkey": 29, "pid": 3, "shipdate": "1992-07-25", "orderkey": 868 }
-{ "partkey": 31, "pid": 1, "shipdate": "1992-07-14", "orderkey": 4705 }
-{ "partkey": 31, "pid": 2, "shipdate": "1992-09-24", "orderkey": 1185 }
-{ "partkey": 31, "pid": 3, "shipdate": "1992-09-29", "orderkey": 5415 }
-{ "partkey": 35, "pid": 1, "shipdate": "1992-03-11", "orderkey": 4230 }
-{ "partkey": 35, "pid": 2, "shipdate": "1992-04-06", "orderkey": 4804 }
-{ "partkey": 35, "pid": 3, "shipdate": "1992-05-26", "orderkey": 2880 }
-{ "partkey": 46, "pid": 1, "shipdate": "1992-04-28", "orderkey": 4230 }
-{ "partkey": 46, "pid": 2, "shipdate": "1992-05-08", "orderkey": 3043 }
-{ "partkey": 46, "pid": 3, "shipdate": "1992-05-21", "orderkey": 3845 }
-{ "partkey": 48, "pid": 1, "shipdate": "1992-05-10", "orderkey": 2691 }
-{ "partkey": 48, "pid": 2, "shipdate": "1992-06-03", "orderkey": 5473 }
-{ "partkey": 48, "pid": 3, "shipdate": "1992-06-15", "orderkey": 832 }
-{ "partkey": 52, "pid": 1, "shipdate": "1992-05-31", "orderkey": 1057 }
-{ "partkey": 52, "pid": 2, "shipdate": "1992-09-03", "orderkey": 4838 }
-{ "partkey": 52, "pid": 3, "shipdate": "1992-09-21", "orderkey": 3907 }
-{ "partkey": 55, "pid": 1, "shipdate": "1992-01-16", "orderkey": 5382 }
-{ "partkey": 55, "pid": 2, "shipdate": "1992-05-11", "orderkey": 1856 }
-{ "partkey": 55, "pid": 3, "shipdate": "1992-06-17", "orderkey": 2022 }
-{ "partkey": 56, "pid": 1, "shipdate": "1992-01-16", "orderkey": 1248 }
-{ "partkey": 56, "pid": 2, "shipdate": "1992-03-02", "orderkey": 3685 }
-{ "partkey": 56, "pid": 3, "shipdate": "1992-06-18", "orderkey": 3205 }
-{ "partkey": 57, "pid": 1, "shipdate": "1992-01-16", "orderkey": 3271 }
-{ "partkey": 57, "pid": 2, "shipdate": "1992-07-06", "orderkey": 194 }
-{ "partkey": 57, "pid": 3, "shipdate": "1992-09-21", "orderkey": 2146 }
-{ "partkey": 59, "pid": 1, "shipdate": "1992-02-09", "orderkey": 2688 }
-{ "partkey": 59, "pid": 2, "shipdate": "1992-03-17", "orderkey": 4998 }
-{ "partkey": 59, "pid": 3, "shipdate": "1992-06-12", "orderkey": 3845 }
-{ "partkey": 62, "pid": 1, "shipdate": "1992-02-01", "orderkey": 1248 }
-{ "partkey": 62, "pid": 2, "shipdate": "1992-03-26", "orderkey": 5382 }
-{ "partkey": 62, "pid": 3, "shipdate": "1992-06-19", "orderkey": 4483 }
-{ "partkey": 63, "pid": 1, "shipdate": "1992-02-07", "orderkey": 4998 }
-{ "partkey": 63, "pid": 2, "shipdate": "1992-06-15", "orderkey": 3650 }
-{ "partkey": 63, "pid": 3, "shipdate": "1993-02-07", "orderkey": 4545 }
-{ "partkey": 64, "pid": 1, "shipdate": "1992-02-13", "orderkey": 2755 }
-{ "partkey": 64, "pid": 2, "shipdate": "1992-02-14", "orderkey": 5409 }
-{ "partkey": 64, "pid": 3, "shipdate": "1992-03-10", "orderkey": 3271 }
-{ "partkey": 65, "pid": 1, "shipdate": "1992-03-02", "orderkey": 4804 }
-{ "partkey": 65, "pid": 2, "shipdate": "1992-04-14", "orderkey": 2848 }
-{ "partkey": 65, "pid": 3, "shipdate": "1992-06-26", "orderkey": 5095 }
-{ "partkey": 68, "pid": 1, "shipdate": "1992-04-13", "orderkey": 3842 }
-{ "partkey": 68, "pid": 2, "shipdate": "1992-06-08", "orderkey": 5121 }
-{ "partkey": 68, "pid": 3, "shipdate": "1992-06-22", "orderkey": 2052 }
-{ "partkey": 71, "pid": 1, "shipdate": "1992-11-10", "orderkey": 2497 }
-{ "partkey": 71, "pid": 2, "shipdate": "1993-01-10", "orderkey": 2146 }
-{ "partkey": 71, "pid": 3, "shipdate": "1993-02-28", "orderkey": 4611 }
-{ "partkey": 73, "pid": 1, "shipdate": "1992-01-08", "orderkey": 5601 }
-{ "partkey": 73, "pid": 2, "shipdate": "1992-09-16", "orderkey": 4741 }
-{ "partkey": 73, "pid": 3, "shipdate": "1993-07-02", "orderkey": 4743 }
-{ "partkey": 75, "pid": 1, "shipdate": "1992-03-27", "orderkey": 4000 }
-{ "partkey": 75, "pid": 2, "shipdate": "1992-05-12", "orderkey": 4230 }
-{ "partkey": 75, "pid": 3, "shipdate": "1992-09-19", "orderkey": 4103 }
-{ "partkey": 81, "pid": 1, "shipdate": "1992-04-11", "orderkey": 2240 }
-{ "partkey": 81, "pid": 2, "shipdate": "1992-06-22", "orderkey": 1221 }
-{ "partkey": 81, "pid": 3, "shipdate": "1992-12-30", "orderkey": 5954 }
-{ "partkey": 91, "pid": 1, "shipdate": "1992-05-22", "orderkey": 3043 }
-{ "partkey": 91, "pid": 2, "shipdate": "1992-06-21", "orderkey": 2691 }
-{ "partkey": 91, "pid": 3, "shipdate": "1992-12-03", "orderkey": 3015 }
-{ "partkey": 97, "pid": 1, "shipdate": "1992-01-27", "orderkey": 4800 }
-{ "partkey": 97, "pid": 2, "shipdate": "1992-03-22", "orderkey": 1856 }
-{ "partkey": 97, "pid": 3, "shipdate": "1992-04-21", "orderkey": 4035 }
-{ "partkey": 98, "pid": 1, "shipdate": "1992-10-06", "orderkey": 5603 }
-{ "partkey": 98, "pid": 2, "shipdate": "1992-12-09", "orderkey": 1159 }
-{ "partkey": 98, "pid": 3, "shipdate": "1993-03-09", "orderkey": 678 }
-{ "partkey": 100, "pid": 1, "shipdate": "1992-03-24", "orderkey": 292 }
-{ "partkey": 100, "pid": 2, "shipdate": "1992-03-24", "orderkey": 2022 }
-{ "partkey": 100, "pid": 3, "shipdate": "1992-06-18", "orderkey": 4738 }
-{ "partkey": 106, "pid": 1, "shipdate": "1992-07-09", "orderkey": 5095 }
-{ "partkey": 106, "pid": 2, "shipdate": "1992-07-31", "orderkey": 3681 }
-{ "partkey": 106, "pid": 3, "shipdate": "1992-10-02", "orderkey": 967 }
-{ "partkey": 114, "pid": 1, "shipdate": "1992-11-19", "orderkey": 3014 }
-{ "partkey": 114, "pid": 2, "shipdate": "1992-11-22", "orderkey": 1506 }
-{ "partkey": 114, "pid": 3, "shipdate": "1993-03-22", "orderkey": 710 }
-{ "partkey": 119, "pid": 1, "shipdate": "1992-05-08", "orderkey": 5574 }
-{ "partkey": 119, "pid": 2, "shipdate": "1992-05-27", "orderkey": 5959 }
-{ "partkey": 119, "pid": 3, "shipdate": "1992-09-07", "orderkey": 4294 }
-{ "partkey": 121, "pid": 1, "shipdate": "1992-04-23", "orderkey": 4903 }
-{ "partkey": 121, "pid": 2, "shipdate": "1992-06-09", "orderkey": 1764 }
-{ "partkey": 121, "pid": 3, "shipdate": "1992-06-23", "orderkey": 2054 }
-{ "partkey": 122, "pid": 1, "shipdate": "1992-03-12", "orderkey": 1248 }
-{ "partkey": 122, "pid": 2, "shipdate": "1992-04-09", "orderkey": 2912 }
-{ "partkey": 122, "pid": 3, "shipdate": "1992-06-05", "orderkey": 801 }
-{ "partkey": 127, "pid": 1, "shipdate": "1992-06-04", "orderkey": 2023 }
-{ "partkey": 127, "pid": 2, "shipdate": "1992-07-02", "orderkey": 37 }
-{ "partkey": 127, "pid": 3, "shipdate": "1994-01-13", "orderkey": 1316 }
-{ "partkey": 129, "pid": 1, "shipdate": "1992-03-31", "orderkey": 2022 }
-{ "partkey": 129, "pid": 2, "shipdate": "1992-05-28", "orderkey": 5953 }
-{ "partkey": 129, "pid": 3, "shipdate": "1992-08-15", "orderkey": 130 }
-{ "partkey": 130, "pid": 1, "shipdate": "1992-04-03", "orderkey": 4705 }
-{ "partkey": 130, "pid": 2, "shipdate": "1992-05-23", "orderkey": 1856 }
-{ "partkey": 130, "pid": 3, "shipdate": "1992-08-20", "orderkey": 644 }
-{ "partkey": 131, "pid": 1, "shipdate": "1992-02-27", "orderkey": 2755 }
-{ "partkey": 131, "pid": 2, "shipdate": "1992-03-03", "orderkey": 4292 }
-{ "partkey": 131, "pid": 3, "shipdate": "1992-05-14", "orderkey": 2627 }
-{ "partkey": 140, "pid": 1, "shipdate": "1992-03-20", "orderkey": 1537 }
-{ "partkey": 140, "pid": 2, "shipdate": "1992-04-27", "orderkey": 6 }
-{ "partkey": 140, "pid": 3, "shipdate": "1992-08-03", "orderkey": 2881 }
-{ "partkey": 143, "pid": 1, "shipdate": "1992-04-17", "orderkey": 1154 }
-{ "partkey": 143, "pid": 2, "shipdate": "1992-09-01", "orderkey": 3524 }
-{ "partkey": 143, "pid": 3, "shipdate": "1992-09-05", "orderkey": 1285 }
-{ "partkey": 144, "pid": 1, "shipdate": "1992-07-05", "orderkey": 4992 }
-{ "partkey": 144, "pid": 2, "shipdate": "1992-08-25", "orderkey": 5415 }
-{ "partkey": 144, "pid": 3, "shipdate": "1992-09-17", "orderkey": 4996 }
-{ "partkey": 149, "pid": 1, "shipdate": "1992-03-22", "orderkey": 5382 }
-{ "partkey": 149, "pid": 2, "shipdate": "1992-04-29", "orderkey": 2688 }
-{ "partkey": 149, "pid": 3, "shipdate": "1992-05-14", "orderkey": 194 }
-{ "partkey": 158, "pid": 1, "shipdate": "1992-08-01", "orderkey": 1955 }
-{ "partkey": 158, "pid": 2, "shipdate": "1992-08-29", "orderkey": 5254 }
-{ "partkey": 158, "pid": 3, "shipdate": "1992-09-18", "orderkey": 5089 }
-{ "partkey": 159, "pid": 1, "shipdate": "1992-05-07", "orderkey": 5409 }
-{ "partkey": 159, "pid": 2, "shipdate": "1992-06-03", "orderkey": 1955 }
-{ "partkey": 159, "pid": 3, "shipdate": "1992-07-10", "orderkey": 4738 }
-{ "partkey": 161, "pid": 1, "shipdate": "1992-03-29", "orderkey": 2240 }
-{ "partkey": 161, "pid": 2, "shipdate": "1992-06-18", "orderkey": 4391 }
-{ "partkey": 161, "pid": 3, "shipdate": "1992-08-28", "orderkey": 5060 }
-{ "partkey": 162, "pid": 1, "shipdate": "1992-04-10", "orderkey": 5953 }
-{ "partkey": 162, "pid": 2, "shipdate": "1992-05-03", "orderkey": 2786 }
-{ "partkey": 162, "pid": 3, "shipdate": "1992-06-11", "orderkey": 2691 }
-{ "partkey": 163, "pid": 1, "shipdate": "1992-02-09", "orderkey": 2983 }
-{ "partkey": 163, "pid": 2, "shipdate": "1992-04-27", "orderkey": 4292 }
-{ "partkey": 163, "pid": 3, "shipdate": "1992-06-01", "orderkey": 4992 }
-{ "partkey": 164, "pid": 1, "shipdate": "1992-03-25", "orderkey": 5601 }
-{ "partkey": 164, "pid": 2, "shipdate": "1992-04-17", "orderkey": 1248 }
-{ "partkey": 164, "pid": 3, "shipdate": "1992-06-06", "orderkey": 801 }
-{ "partkey": 165, "pid": 1, "shipdate": "1992-03-21", "orderkey": 2848 }
-{ "partkey": 165, "pid": 2, "shipdate": "1992-04-01", "orderkey": 4903 }
-{ "partkey": 165, "pid": 3, "shipdate": "1992-04-12", "orderkey": 3168 }
-{ "partkey": 168, "pid": 1, "shipdate": "1992-05-06", "orderkey": 194 }
-{ "partkey": 168, "pid": 2, "shipdate": "1992-07-20", "orderkey": 3654 }
-{ "partkey": 168, "pid": 3, "shipdate": "1992-10-07", "orderkey": 868 }
-{ "partkey": 172, "pid": 1, "shipdate": "1992-09-06", "orderkey": 2247 }
-{ "partkey": 172, "pid": 2, "shipdate": "1993-05-01", "orderkey": 167 }
-{ "partkey": 172, "pid": 3, "shipdate": "1993-06-16", "orderkey": 1600 }
-{ "partkey": 175, "pid": 1, "shipdate": "1992-10-09", "orderkey": 929 }
-{ "partkey": 175, "pid": 2, "shipdate": "1992-11-09", "orderkey": 4294 }
-{ "partkey": 175, "pid": 3, "shipdate": "1992-11-10", "orderkey": 4261 }
-{ "partkey": 177, "pid": 1, "shipdate": "1992-04-05", "orderkey": 5382 }
-{ "partkey": 177, "pid": 2, "shipdate": "1992-12-25", "orderkey": 1956 }
-{ "partkey": 177, "pid": 3, "shipdate": "1993-01-16", "orderkey": 3680 }
-{ "partkey": 181, "pid": 1, "shipdate": "1992-07-01", "orderkey": 1088 }
-{ "partkey": 181, "pid": 2, "shipdate": "1992-11-04", "orderkey": 2209 }
-{ "partkey": 181, "pid": 3, "shipdate": "1992-12-14", "orderkey": 3232 }
-{ "partkey": 184, "pid": 1, "shipdate": "1992-04-12", "orderkey": 1925 }
-{ "partkey": 184, "pid": 2, "shipdate": "1992-04-12", "orderkey": 322 }
-{ "partkey": 184, "pid": 3, "shipdate": "1992-04-30", "orderkey": 194 }
-{ "partkey": 186, "pid": 1, "shipdate": "1992-07-26", "orderkey": 4069 }
-{ "partkey": 186, "pid": 2, "shipdate": "1992-11-25", "orderkey": 129 }
-{ "partkey": 186, "pid": 3, "shipdate": "1992-11-27", "orderkey": 481 }
-{ "partkey": 189, "pid": 1, "shipdate": "1992-06-16", "orderkey": 4805 }
-{ "partkey": 189, "pid": 2, "shipdate": "1992-06-20", "orderkey": 134 }
-{ "partkey": 189, "pid": 3, "shipdate": "1992-07-20", "orderkey": 1285 }
-{ "partkey": 194, "pid": 1, "shipdate": "1992-02-14", "orderkey": 5409 }
-{ "partkey": 194, "pid": 2, "shipdate": "1992-06-20", "orderkey": 3842 }
-{ "partkey": 194, "pid": 3, "shipdate": "1992-12-15", "orderkey": 5062 }
-{ "partkey": 197, "pid": 1, "shipdate": "1993-08-22", "orderkey": 2080 }
-{ "partkey": 197, "pid": 2, "shipdate": "1994-02-24", "orderkey": 3138 }
-{ "partkey": 197, "pid": 3, "shipdate": "1994-03-03", "orderkey": 70 }
-{ "partkey": 198, "pid": 1, "shipdate": "1992-04-21", "orderkey": 3011 }
-{ "partkey": 198, "pid": 2, "shipdate": "1992-09-12", "orderkey": 4294 }
-{ "partkey": 198, "pid": 3, "shipdate": "1992-12-27", "orderkey": 1345 }
-{ "partkey": 8, "pid": 1, "shipdate": "1992-09-25", "orderkey": 5635 }
-{ "partkey": 8, "pid": 2, "shipdate": "1992-11-15", "orderkey": 1540 }
-{ "partkey": 8, "pid": 3, "shipdate": "1993-02-13", "orderkey": 1222 }
-{ "partkey": 9, "pid": 1, "shipdate": "1992-04-29", "orderkey": 3970 }
-{ "partkey": 9, "pid": 2, "shipdate": "1992-04-30", "orderkey": 1955 }
-{ "partkey": 9, "pid": 3, "shipdate": "1992-06-01", "orderkey": 4199 }
-{ "partkey": 10, "pid": 1, "shipdate": "1992-05-13", "orderkey": 2881 }
-{ "partkey": 10, "pid": 2, "shipdate": "1992-11-25", "orderkey": 5378 }
-{ "partkey": 10, "pid": 3, "shipdate": "1992-12-01", "orderkey": 1796 }
-{ "partkey": 25, "pid": 1, "shipdate": "1992-02-04", "orderkey": 2688 }
-{ "partkey": 25, "pid": 2, "shipdate": "1992-07-23", "orderkey": 5060 }
-{ "partkey": 25, "pid": 3, "shipdate": "1992-08-01", "orderkey": 868 }
-{ "partkey": 28, "pid": 1, "shipdate": "1992-03-16", "orderkey": 2240 }
-{ "partkey": 28, "pid": 2, "shipdate": "1992-10-13", "orderkey": 5699 }
-{ "partkey": 28, "pid": 3, "shipdate": "1992-11-04", "orderkey": 1506 }
-{ "partkey": 34, "pid": 1, "shipdate": "1992-07-03", "orderkey": 322 }
-{ "partkey": 34, "pid": 2, "shipdate": "1992-07-20", "orderkey": 3845 }
-{ "partkey": 34, "pid": 3, "shipdate": "1992-11-23", "orderkey": 5089 }
-{ "partkey": 37, "pid": 1, "shipdate": "1992-08-30", "orderkey": 1088 }
-{ "partkey": 37, "pid": 2, "shipdate": "1992-10-03", "orderkey": 2500 }
-{ "partkey": 37, "pid": 3, "shipdate": "1993-01-31", "orderkey": 3074 }
-{ "partkey": 40, "pid": 1, "shipdate": "1992-02-07", "orderkey": 4292 }
-{ "partkey": 40, "pid": 2, "shipdate": "1992-04-28", "orderkey": 3139 }
-{ "partkey": 40, "pid": 3, "shipdate": "1992-05-03", "orderkey": 3973 }
-{ "partkey": 41, "pid": 1, "shipdate": "1992-12-13", "orderkey": 4896 }
-{ "partkey": 41, "pid": 2, "shipdate": "1993-01-18", "orderkey": 2852 }
-{ "partkey": 41, "pid": 3, "shipdate": "1993-04-13", "orderkey": 3367 }
-{ "partkey": 43, "pid": 1, "shipdate": "1992-06-18", "orderkey": 4069 }
-{ "partkey": 43, "pid": 2, "shipdate": "1992-06-30", "orderkey": 2052 }
-{ "partkey": 43, "pid": 3, "shipdate": "1992-08-28", "orderkey": 5959 }
-{ "partkey": 66, "pid": 1, "shipdate": "1992-05-07", "orderkey": 194 }
-{ "partkey": 66, "pid": 2, "shipdate": "1992-09-11", "orderkey": 549 }
-{ "partkey": 66, "pid": 3, "shipdate": "1992-10-10", "orderkey": 3015 }
-{ "partkey": 67, "pid": 1, "shipdate": "1992-05-13", "orderkey": 1764 }
-{ "partkey": 67, "pid": 2, "shipdate": "1993-01-08", "orderkey": 612 }
-{ "partkey": 67, "pid": 3, "shipdate": "1993-11-03", "orderkey": 2631 }
-{ "partkey": 69, "pid": 1, "shipdate": "1992-05-31", "orderkey": 3205 }
-{ "partkey": 69, "pid": 2, "shipdate": "1992-06-05", "orderkey": 5767 }
-{ "partkey": 69, "pid": 3, "shipdate": "1992-07-01", "orderkey": 1221 }
-{ "partkey": 76, "pid": 1, "shipdate": "1992-10-22", "orderkey": 5408 }
-{ "partkey": 76, "pid": 2, "shipdate": "1993-04-19", "orderkey": 2272 }
-{ "partkey": 76, "pid": 3, "shipdate": "1993-06-12", "orderkey": 2245 }
-{ "partkey": 79, "pid": 1, "shipdate": "1992-08-05", "orderkey": 4069 }
-{ "partkey": 79, "pid": 2, "shipdate": "1992-08-10", "orderkey": 5986 }
-{ "partkey": 79, "pid": 3, "shipdate": "1993-04-08", "orderkey": 4418 }
-{ "partkey": 84, "pid": 1, "shipdate": "1992-09-08", "orderkey": 1285 }
-{ "partkey": 84, "pid": 2, "shipdate": "1993-05-15", "orderkey": 2597 }
-{ "partkey": 84, "pid": 3, "shipdate": "1993-05-20", "orderkey": 772 }
-{ "partkey": 85, "pid": 1, "shipdate": "1992-02-28", "orderkey": 1057 }
-{ "partkey": 85, "pid": 2, "shipdate": "1992-05-28", "orderkey": 5574 }
-{ "partkey": 85, "pid": 3, "shipdate": "1992-06-27", "orderkey": 1221 }
-{ "partkey": 86, "pid": 1, "shipdate": "1992-05-25", "orderkey": 2240 }
-{ "partkey": 86, "pid": 2, "shipdate": "1992-11-18", "orderkey": 4896 }
-{ "partkey": 86, "pid": 3, "shipdate": "1993-03-01", "orderkey": 4166 }
-{ "partkey": 94, "pid": 1, "shipdate": "1992-05-20", "orderkey": 5574 }
-{ "partkey": 94, "pid": 2, "shipdate": "1992-07-03", "orderkey": 3650 }
-{ "partkey": 94, "pid": 3, "shipdate": "1992-07-26", "orderkey": 3654 }
-{ "partkey": 95, "pid": 1, "shipdate": "1992-02-24", "orderkey": 3271 }
-{ "partkey": 95, "pid": 2, "shipdate": "1992-03-14", "orderkey": 801 }
-{ "partkey": 95, "pid": 3, "shipdate": "1992-11-17", "orderkey": 2176 }
-{ "partkey": 96, "pid": 1, "shipdate": "1992-06-18", "orderkey": 2052 }
-{ "partkey": 96, "pid": 2, "shipdate": "1992-09-26", "orderkey": 3172 }
-{ "partkey": 96, "pid": 3, "shipdate": "1992-11-25", "orderkey": 1159 }
-{ "partkey": 99, "pid": 1, "shipdate": "1992-05-01", "orderkey": 4998 }
-{ "partkey": 99, "pid": 2, "shipdate": "1993-04-18", "orderkey": 1409 }
-{ "partkey": 99, "pid": 3, "shipdate": "1993-06-09", "orderkey": 2149 }
-{ "partkey": 102, "pid": 1, "shipdate": "1992-08-19", "orderkey": 5415 }
-{ "partkey": 102, "pid": 2, "shipdate": "1992-08-21", "orderkey": 5408 }
-{ "partkey": 102, "pid": 3, "shipdate": "1992-10-25", "orderkey": 674 }
-{ "partkey": 108, "pid": 1, "shipdate": "1992-07-28", "orderkey": 1826 }
-{ "partkey": 108, "pid": 2, "shipdate": "1992-08-01", "orderkey": 1221 }
-{ "partkey": 108, "pid": 3, "shipdate": "1992-09-07", "orderkey": 2560 }
-{ "partkey": 111, "pid": 1, "shipdate": "1992-07-05", "orderkey": 4705 }
-{ "partkey": 111, "pid": 2, "shipdate": "1992-07-28", "orderkey": 5254 }
-{ "partkey": 111, "pid": 3, "shipdate": "1992-08-13", "orderkey": 5121 }
-{ "partkey": 113, "pid": 1, "shipdate": "1992-06-08", "orderkey": 1027 }
-{ "partkey": 113, "pid": 2, "shipdate": "1992-08-13", "orderkey": 2054 }
-{ "partkey": 113, "pid": 3, "shipdate": "1992-08-25", "orderkey": 4741 }
-{ "partkey": 116, "pid": 1, "shipdate": "1992-03-22", "orderkey": 2755 }
-{ "partkey": 116, "pid": 2, "shipdate": "1992-05-17", "orderkey": 1925 }
-{ "partkey": 116, "pid": 3, "shipdate": "1992-06-24", "orderkey": 5603 }
-{ "partkey": 124, "pid": 1, "shipdate": "1992-06-15", "orderkey": 1088 }
-{ "partkey": 124, "pid": 2, "shipdate": "1992-08-09", "orderkey": 2209 }
-{ "partkey": 124, "pid": 3, "shipdate": "1992-09-13", "orderkey": 1346 }
-{ "partkey": 128, "pid": 1, "shipdate": "1992-03-05", "orderkey": 3168 }
-{ "partkey": 128, "pid": 2, "shipdate": "1992-05-02", "orderkey": 4804 }
-{ "partkey": 128, "pid": 3, "shipdate": "1992-08-24", "orderkey": 4096 }
-{ "partkey": 132, "pid": 1, "shipdate": "1992-04-17", "orderkey": 5607 }
-{ "partkey": 132, "pid": 2, "shipdate": "1992-06-14", "orderkey": 384 }
-{ "partkey": 132, "pid": 3, "shipdate": "1992-07-06", "orderkey": 3172 }
-{ "partkey": 133, "pid": 1, "shipdate": "1992-06-08", "orderkey": 3140 }
-{ "partkey": 133, "pid": 2, "shipdate": "1992-11-17", "orderkey": 4864 }
-{ "partkey": 133, "pid": 3, "shipdate": "1993-01-18", "orderkey": 1506 }
-{ "partkey": 134, "pid": 1, "shipdate": "1992-05-17", "orderkey": 3685 }
-{ "partkey": 134, "pid": 2, "shipdate": "1992-05-20", "orderkey": 644 }
-{ "partkey": 134, "pid": 3, "shipdate": "1992-05-29", "orderkey": 421 }
-{ "partkey": 136, "pid": 1, "shipdate": "1992-05-19", "orderkey": 2786 }
-{ "partkey": 136, "pid": 2, "shipdate": "1992-05-21", "orderkey": 4035 }
-{ "partkey": 136, "pid": 3, "shipdate": "1992-06-07", "orderkey": 4805 }
-{ "partkey": 139, "pid": 1, "shipdate": "1992-04-12", "orderkey": 2880 }
-{ "partkey": 139, "pid": 2, "shipdate": "1992-06-28", "orderkey": 4992 }
-{ "partkey": 139, "pid": 3, "shipdate": "1992-09-12", "orderkey": 4099 }
-{ "partkey": 150, "pid": 1, "shipdate": "1992-05-01", "orderkey": 4805 }
-{ "partkey": 150, "pid": 2, "shipdate": "1992-05-02", "orderkey": 1856 }
-{ "partkey": 150, "pid": 3, "shipdate": "1992-05-25", "orderkey": 1701 }
-{ "partkey": 155, "pid": 1, "shipdate": "1992-09-28", "orderkey": 1956 }
-{ "partkey": 155, "pid": 2, "shipdate": "1992-11-25", "orderkey": 5378 }
-{ "partkey": 155, "pid": 3, "shipdate": "1993-05-14", "orderkey": 2305 }
-{ "partkey": 160, "pid": 1, "shipdate": "1992-05-07", "orderkey": 1282 }
-{ "partkey": 160, "pid": 2, "shipdate": "1992-07-04", "orderkey": 4867 }
-{ "partkey": 160, "pid": 3, "shipdate": "1992-08-18", "orderkey": 1346 }
-{ "partkey": 166, "pid": 1, "shipdate": "1992-08-11", "orderkey": 2691 }
-{ "partkey": 166, "pid": 2, "shipdate": "1992-08-14", "orderkey": 5095 }
-{ "partkey": 166, "pid": 3, "shipdate": "1993-04-22", "orderkey": 1703 }
-{ "partkey": 167, "pid": 1, "shipdate": "1992-06-02", "orderkey": 5767 }
-{ "partkey": 167, "pid": 2, "shipdate": "1993-01-31", "orderkey": 1447 }
-{ "partkey": 167, "pid": 3, "shipdate": "1993-02-15", "orderkey": 1857 }
-{ "partkey": 174, "pid": 1, "shipdate": "1992-06-25", "orderkey": 2054 }
-{ "partkey": 174, "pid": 2, "shipdate": "1992-11-02", "orderkey": 1991 }
-{ "partkey": 174, "pid": 3, "shipdate": "1992-12-02", "orderkey": 4261 }
-{ "partkey": 183, "pid": 1, "shipdate": "1992-04-24", "orderkey": 4998 }
-{ "partkey": 183, "pid": 2, "shipdate": "1992-10-24", "orderkey": 5408 }
-{ "partkey": 183, "pid": 3, "shipdate": "1993-01-08", "orderkey": 1571 }
-{ "partkey": 185, "pid": 1, "shipdate": "1992-04-30", "orderkey": 3712 }
-{ "partkey": 185, "pid": 2, "shipdate": "1992-06-20", "orderkey": 5574 }
-{ "partkey": 185, "pid": 3, "shipdate": "1992-07-23", "orderkey": 2023 }
-{ "partkey": 187, "pid": 1, "shipdate": "1992-04-01", "orderkey": 4391 }
-{ "partkey": 187, "pid": 2, "shipdate": "1992-05-30", "orderkey": 4738 }
-{ "partkey": 187, "pid": 3, "shipdate": "1992-06-01", "orderkey": 4738 }
-{ "partkey": 191, "pid": 1, "shipdate": "1992-07-31", "orderkey": 5767 }
-{ "partkey": 191, "pid": 2, "shipdate": "1992-08-29", "orderkey": 3361 }
-{ "partkey": 191, "pid": 3, "shipdate": "1992-09-22", "orderkey": 1506 }
-{ "partkey": 192, "pid": 1, "shipdate": "1992-02-19", "orderkey": 3685 }
-{ "partkey": 192, "pid": 2, "shipdate": "1992-08-10", "orderkey": 5254 }
-{ "partkey": 192, "pid": 3, "shipdate": "1992-09-02", "orderkey": 2500 }
-{ "partkey": 196, "pid": 1, "shipdate": "1992-03-02", "orderkey": 4000 }
-{ "partkey": 196, "pid": 2, "shipdate": "1992-03-04", "orderkey": 1154 }
-{ "partkey": 196, "pid": 3, "shipdate": "1992-06-11", "orderkey": 4230 }
 { "partkey": 200, "pid": 1, "shipdate": "1992-04-19", "orderkey": 324 }
 { "partkey": 200, "pid": 2, "shipdate": "1993-01-06", "orderkey": 1447 }
 { "partkey": 200, "pid": 3, "shipdate": "1993-10-17", "orderkey": 5764 }
-{ "partkey": 3, "pid": 1, "shipdate": "1992-04-25", "orderkey": 801 }
-{ "partkey": 3, "pid": 2, "shipdate": "1992-05-24", "orderkey": 194 }
-{ "partkey": 3, "pid": 3, "shipdate": "1993-01-03", "orderkey": 3776 }
-{ "partkey": 5, "pid": 1, "shipdate": "1992-05-02", "orderkey": 3970 }
-{ "partkey": 5, "pid": 2, "shipdate": "1992-06-14", "orderkey": 5959 }
-{ "partkey": 5, "pid": 3, "shipdate": "1993-01-06", "orderkey": 3680 }
-{ "partkey": 7, "pid": 1, "shipdate": "1992-04-12", "orderkey": 3140 }
-{ "partkey": 7, "pid": 2, "shipdate": "1993-02-11", "orderkey": 3204 }
-{ "partkey": 7, "pid": 3, "shipdate": "1993-06-25", "orderkey": 5794 }
-{ "partkey": 17, "pid": 1, "shipdate": "1992-07-23", "orderkey": 967 }
-{ "partkey": 17, "pid": 2, "shipdate": "1993-03-01", "orderkey": 931 }
-{ "partkey": 17, "pid": 3, "shipdate": "1993-05-06", "orderkey": 611 }
-{ "partkey": 18, "pid": 1, "shipdate": "1992-04-12", "orderkey": 1537 }
-{ "partkey": 18, "pid": 2, "shipdate": "1992-04-21", "orderkey": 2880 }
-{ "partkey": 18, "pid": 3, "shipdate": "1992-05-21", "orderkey": 2688 }
-{ "partkey": 27, "pid": 1, "shipdate": "1992-07-05", "orderkey": 1826 }
-{ "partkey": 27, "pid": 2, "shipdate": "1992-07-14", "orderkey": 4096 }
-{ "partkey": 27, "pid": 3, "shipdate": "1992-08-17", "orderkey": 4294 }
-{ "partkey": 32, "pid": 1, "shipdate": "1992-09-22", "orderkey": 4900 }
-{ "partkey": 32, "pid": 2, "shipdate": "1992-09-25", "orderkey": 5060 }
-{ "partkey": 32, "pid": 3, "shipdate": "1992-10-07", "orderkey": 5603 }
-{ "partkey": 36, "pid": 1, "shipdate": "1992-02-26", "orderkey": 1154 }
-{ "partkey": 36, "pid": 2, "shipdate": "1992-07-03", "orderkey": 134 }
-{ "partkey": 36, "pid": 3, "shipdate": "1993-01-06", "orderkey": 3521 }
-{ "partkey": 39, "pid": 1, "shipdate": "1992-05-26", "orderkey": 4515 }
-{ "partkey": 39, "pid": 2, "shipdate": "1992-11-12", "orderkey": 612 }
-{ "partkey": 39, "pid": 3, "shipdate": "1992-11-15", "orderkey": 1447 }
-{ "partkey": 42, "pid": 1, "shipdate": "1992-10-23", "orderkey": 2560 }
-{ "partkey": 42, "pid": 2, "shipdate": "1992-11-04", "orderkey": 2566 }
-{ "partkey": 42, "pid": 3, "shipdate": "1992-12-12", "orderkey": 1571 }
-{ "partkey": 44, "pid": 1, "shipdate": "1992-02-14", "orderkey": 4292 }
-{ "partkey": 44, "pid": 2, "shipdate": "1992-06-11", "orderkey": 322 }
-{ "partkey": 44, "pid": 3, "shipdate": "1992-11-29", "orderkey": 2147 }
-{ "partkey": 50, "pid": 1, "shipdate": "1992-04-22", "orderkey": 2786 }
-{ "partkey": 50, "pid": 2, "shipdate": "1992-07-31", "orderkey": 644 }
-{ "partkey": 50, "pid": 3, "shipdate": "1992-09-23", "orderkey": 2885 }
-{ "partkey": 53, "pid": 1, "shipdate": "1992-01-14", "orderkey": 4800 }
-{ "partkey": 53, "pid": 2, "shipdate": "1992-05-22", "orderkey": 2240 }
-{ "partkey": 53, "pid": 3, "shipdate": "1992-10-04", "orderkey": 2562 }
-{ "partkey": 54, "pid": 1, "shipdate": "1992-04-07", "orderkey": 4515 }
-{ "partkey": 54, "pid": 2, "shipdate": "1992-05-01", "orderkey": 3271 }
-{ "partkey": 54, "pid": 3, "shipdate": "1992-06-24", "orderkey": 1701 }
-{ "partkey": 61, "pid": 1, "shipdate": "1993-07-14", "orderkey": 2020 }
-{ "partkey": 61, "pid": 2, "shipdate": "1993-07-15", "orderkey": 5318 }
-{ "partkey": 61, "pid": 3, "shipdate": "1993-09-29", "orderkey": 261 }
-{ "partkey": 74, "pid": 1, "shipdate": "1992-03-21", "orderkey": 4162 }
-{ "partkey": 74, "pid": 2, "shipdate": "1992-03-22", "orderkey": 801 }
-{ "partkey": 74, "pid": 3, "shipdate": "1992-10-21", "orderkey": 929 }
-{ "partkey": 78, "pid": 1, "shipdate": "1992-03-04", "orderkey": 2210 }
-{ "partkey": 78, "pid": 2, "shipdate": "1992-04-04", "orderkey": 2022 }
-{ "partkey": 78, "pid": 3, "shipdate": "1992-05-06", "orderkey": 1764 }
-{ "partkey": 80, "pid": 1, "shipdate": "1992-05-18", "orderkey": 644 }
-{ "partkey": 80, "pid": 2, "shipdate": "1992-09-02", "orderkey": 2500 }
-{ "partkey": 80, "pid": 3, "shipdate": "1993-06-07", "orderkey": 3877 }
-{ "partkey": 82, "pid": 1, "shipdate": "1992-07-17", "orderkey": 4867 }
-{ "partkey": 82, "pid": 2, "shipdate": "1992-10-18", "orderkey": 1504 }
-{ "partkey": 82, "pid": 3, "shipdate": "1992-12-11", "orderkey": 4261 }
-{ "partkey": 83, "pid": 1, "shipdate": "1992-06-09", "orderkey": 4738 }
-{ "partkey": 83, "pid": 2, "shipdate": "1992-08-04", "orderkey": 5218 }
-{ "partkey": 83, "pid": 3, "shipdate": "1992-09-21", "orderkey": 5220 }
-{ "partkey": 87, "pid": 1, "shipdate": "1992-09-30", "orderkey": 4294 }
-{ "partkey": 87, "pid": 2, "shipdate": "1992-12-02", "orderkey": 1540 }
-{ "partkey": 87, "pid": 3, "shipdate": "1993-01-06", "orderkey": 3556 }
-{ "partkey": 90, "pid": 1, "shipdate": "1992-02-25", "orderkey": 4162 }
-{ "partkey": 90, "pid": 2, "shipdate": "1992-06-07", "orderkey": 5474 }
-{ "partkey": 90, "pid": 3, "shipdate": "1992-08-21", "orderkey": 5986 }
-{ "partkey": 101, "pid": 1, "shipdate": "1992-08-17", "orderkey": 644 }
-{ "partkey": 101, "pid": 2, "shipdate": "1992-09-27", "orderkey": 2147 }
-{ "partkey": 101, "pid": 3, "shipdate": "1992-12-28", "orderkey": 1571 }
-{ "partkey": 104, "pid": 1, "shipdate": "1992-03-17", "orderkey": 5409 }
-{ "partkey": 104, "pid": 2, "shipdate": "1992-11-08", "orderkey": 4897 }
-{ "partkey": 104, "pid": 3, "shipdate": "1994-01-22", "orderkey": 5479 }
-{ "partkey": 107, "pid": 1, "shipdate": "1992-05-22", "orderkey": 1088 }
-{ "partkey": 107, "pid": 2, "shipdate": "1992-07-30", "orderkey": 3654 }
-{ "partkey": 107, "pid": 3, "shipdate": "1992-08-05", "orderkey": 3842 }
-{ "partkey": 110, "pid": 1, "shipdate": "1992-09-18", "orderkey": 3907 }
-{ "partkey": 110, "pid": 2, "shipdate": "1992-11-01", "orderkey": 4261 }
-{ "partkey": 110, "pid": 3, "shipdate": "1993-01-01", "orderkey": 1991 }
-{ "partkey": 112, "pid": 1, "shipdate": "1992-09-13", "orderkey": 3907 }
-{ "partkey": 112, "pid": 2, "shipdate": "1992-10-09", "orderkey": 2885 }
-{ "partkey": 112, "pid": 3, "shipdate": "1993-01-15", "orderkey": 481 }
-{ "partkey": 117, "pid": 1, "shipdate": "1992-05-04", "orderkey": 1856 }
-{ "partkey": 117, "pid": 2, "shipdate": "1993-03-18", "orderkey": 4545 }
-{ "partkey": 117, "pid": 3, "shipdate": "1993-07-10", "orderkey": 3494 }
-{ "partkey": 120, "pid": 1, "shipdate": "1992-03-23", "orderkey": 4292 }
-{ "partkey": 120, "pid": 2, "shipdate": "1992-04-28", "orderkey": 1221 }
-{ "partkey": 120, "pid": 3, "shipdate": "1992-06-29", "orderkey": 4903 }
-{ "partkey": 123, "pid": 1, "shipdate": "1992-02-01", "orderkey": 3011 }
-{ "partkey": 123, "pid": 2, "shipdate": "1992-06-20", "orderkey": 5095 }
-{ "partkey": 123, "pid": 3, "shipdate": "1992-11-22", "orderkey": 1505 }
-{ "partkey": 137, "pid": 1, "shipdate": "1992-05-23", "orderkey": 3524 }
-{ "partkey": 137, "pid": 2, "shipdate": "1992-07-05", "orderkey": 1955 }
-{ "partkey": 137, "pid": 3, "shipdate": "1992-09-12", "orderkey": 4099 }
-{ "partkey": 142, "pid": 1, "shipdate": "1992-10-14", "orderkey": 3556 }
-{ "partkey": 142, "pid": 2, "shipdate": "1993-05-14", "orderkey": 2241 }
-{ "partkey": 142, "pid": 3, "shipdate": "1993-07-11", "orderkey": 5670 }
-{ "partkey": 145, "pid": 1, "shipdate": "1992-01-25", "orderkey": 4998 }
-{ "partkey": 145, "pid": 2, "shipdate": "1992-08-16", "orderkey": 134 }
-{ "partkey": 145, "pid": 3, "shipdate": "1992-10-25", "orderkey": 3907 }
-{ "partkey": 146, "pid": 1, "shipdate": "1992-05-21", "orderkey": 194 }
-{ "partkey": 146, "pid": 2, "shipdate": "1993-06-21", "orderkey": 678 }
-{ "partkey": 146, "pid": 3, "shipdate": "1993-08-02", "orderkey": 1286 }
-{ "partkey": 148, "pid": 1, "shipdate": "1992-01-15", "orderkey": 3712 }
-{ "partkey": 148, "pid": 2, "shipdate": "1992-02-27", "orderkey": 5601 }
-{ "partkey": 148, "pid": 3, "shipdate": "1992-04-22", "orderkey": 1154 }
-{ "partkey": 151, "pid": 1, "shipdate": "1992-01-26", "orderkey": 1248 }
-{ "partkey": 151, "pid": 2, "shipdate": "1992-07-30", "orderkey": 4256 }
-{ "partkey": 151, "pid": 3, "shipdate": "1992-12-19", "orderkey": 3014 }
-{ "partkey": 169, "pid": 1, "shipdate": "1992-03-31", "orderkey": 1057 }
-{ "partkey": 169, "pid": 2, "shipdate": "1992-06-05", "orderkey": 5953 }
-{ "partkey": 169, "pid": 3, "shipdate": "1992-06-07", "orderkey": 1894 }
-{ "partkey": 173, "pid": 1, "shipdate": "1992-06-17", "orderkey": 4738 }
-{ "partkey": 173, "pid": 2, "shipdate": "1992-09-15", "orderkey": 3654 }
-{ "partkey": 173, "pid": 3, "shipdate": "1992-09-30", "orderkey": 1540 }
-{ "partkey": 176, "pid": 1, "shipdate": "1992-02-01", "orderkey": 4800 }
-{ "partkey": 176, "pid": 2, "shipdate": "1992-04-28", "orderkey": 1826 }
-{ "partkey": 176, "pid": 3, "shipdate": "1992-09-24", "orderkey": 3907 }
-{ "partkey": 188, "pid": 1, "shipdate": "1992-09-15", "orderkey": 1285 }
-{ "partkey": 188, "pid": 2, "shipdate": "1993-04-08", "orderkey": 5381 }
-{ "partkey": 188, "pid": 3, "shipdate": "1993-05-03", "orderkey": 4226 }
-{ "partkey": 193, "pid": 1, "shipdate": "1992-05-05", "orderkey": 1057 }
-{ "partkey": 193, "pid": 2, "shipdate": "1992-08-21", "orderkey": 5795 }
-{ "partkey": 193, "pid": 3, "shipdate": "1993-02-12", "orderkey": 2244 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/group-by.26.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/group-by.26.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/group-by.26.adm
index 53fa5af..3f50b9b 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/group-by.26.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tinysocial/tinysocial-suite-open/group-by.26.adm
@@ -1,10 +1,10 @@
-{ "average": 2009.0, "count": 1, "sum": 2009, "organization": "Zuncan", "min": date("2009-12-13"), "max": date("2009-12-13") }
-{ "average": null, "count": 0, "sum": null, "organization": "Labzatron", "min": null, "max": null }
-{ "average": null, "count": 0, "sum": null, "organization": "Plexlane", "min": null, "max": null }
-{ "average": null, "count": 0, "sum": null, "organization": "jaydax", "min": null, "max": null }
 { "average": null, "count": 0, "sum": null, "organization": "Codetechno", "min": null, "max": null }
 { "average": null, "count": 0, "sum": null, "organization": "Hexviafind", "min": null, "max": null }
-{ "average": null, "count": 0, "sum": null, "organization": "Zamcorporation", "min": null, "max": null }
-{ "average": 2011.0, "count": 1, "sum": 2011, "organization": "physcane", "min": date("2011-11-05"), "max": date("2011-11-05") }
 { "average": null, "count": 0, "sum": null, "organization": "Kongreen", "min": null, "max": null }
+{ "average": null, "count": 0, "sum": null, "organization": "Labzatron", "min": null, "max": null }
+{ "average": null, "count": 0, "sum": null, "organization": "Plexlane", "min": null, "max": null }
+{ "average": null, "count": 0, "sum": null, "organization": "Zamcorporation", "min": null, "max": null }
+{ "average": 2009.0, "count": 1, "sum": 2009, "organization": "Zuncan", "min": date("2009-12-13"), "max": date("2009-12-13") }
 { "average": 2010.0, "count": 1, "sum": 2010, "organization": "geomedia", "min": date("2010-01-26"), "max": date("2010-01-26") }
+{ "average": null, "count": 0, "sum": null, "organization": "jaydax", "min": null, "max": null }
+{ "average": 2011.0, "count": 1, "sum": 2011, "organization": "physcane", "min": date("2011-11-05"), "max": date("2011-11-05") }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue601/query-issue601.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue601/query-issue601.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue601/query-issue601.1.adm
index 7d83268..4c3d838 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue601/query-issue601.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue601/query-issue601.1.adm
@@ -1,7 +1,7 @@
-{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 1, "count_order": 1500 }
 { "l_linenumber": 2, "count_order": 1291 }
-{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 3, "count_order": 1077 }
+{ "l_linenumber": 4, "count_order": 862 }
 { "l_linenumber": 5, "count_order": 632 }
+{ "l_linenumber": 6, "count_order": 432 }
 { "l_linenumber": 7, "count_order": 211 }

http://git-wip-us.apache.org/repos/asf/asterixdb/blob/03e940a7/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785-2/query-issue785-2.1.adm
----------------------------------------------------------------------
diff --git a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785-2/query-issue785-2.1.adm b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785-2/query-issue785-2.1.adm
index 360adf4..3d98aaf 100644
--- a/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785-2/query-issue785-2.1.adm
+++ b/asterixdb/asterix-app/src/test/resources/runtimets/results/tpch-sql-like/query-issue785-2/query-issue785-2.1.adm
@@ -1,10 +1,10 @@
-{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
-{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }
+{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
 { "nation_key": 1, "sum_price": [ { "orderdate": "1993-05-26", "sum_price": 221036.31 }, { "orderdate": "1992-03-20", "sum_price": 216230.27000000002 }, { "orderdate": "1993-12-24", "sum_price": 211925.95 } ] }
 { "nation_key": 2, "sum_price": [ { "orderdate": "1996-03-01", "sum_price": 218697.85 }, { "orderdate": "1996-08-13", "sum_price": 217709.03 }, { "orderdate": "1992-08-21", "sum_price": 207364.8 } ] }
+{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
 { "nation_key": 4, "sum_price": [ { "orderdate": "1993-09-20", "sum_price": 226806.66 }, { "orderdate": "1992-03-04", "sum_price": 219709.6 }, { "orderdate": "1996-01-06", "sum_price": 190490.78 } ] }
 { "nation_key": 19, "sum_price": [ { "orderdate": "1993-12-29", "sum_price": 328959.87 }, { "orderdate": "1997-08-04", "sum_price": 244636.7 }, { "orderdate": "1996-11-20", "sum_price": 222274.54 } ] }
 { "nation_key": 20, "sum_price": [ { "orderdate": "1993-01-31", "sum_price": 190960.69 }, { "orderdate": "1998-07-17", "sum_price": 187156.38 }, { "orderdate": "1993-03-25", "sum_price": 167017.39 } ] }
+{ "nation_key": 21, "sum_price": [ { "orderdate": "1994-02-27", "sum_price": 198360.22 }, { "orderdate": "1992-07-07", "sum_price": 180692.9 }, { "orderdate": "1996-06-28", "sum_price": 139915.23 } ] }
 { "nation_key": 22, "sum_price": [ { "orderdate": "1998-02-27", "sum_price": 263411.29 }, { "orderdate": "1993-04-11", "sum_price": 221636.83 }, { "orderdate": "1993-05-07", "sum_price": 220715.14 } ] }
-{ "nation_key": 0, "sum_price": [ { "orderdate": "1997-01-13", "sum_price": 241837.88 }, { "orderdate": "1997-01-21", "sum_price": 240284.95 }, { "orderdate": "1997-08-24", "sum_price": 231831.35 } ] }
-{ "nation_key": 3, "sum_price": [ { "orderdate": "1997-04-23", "sum_price": 351762.82999999996 }, { "orderdate": "1995-11-13", "sum_price": 242588.87 }, { "orderdate": "1993-07-15", "sum_price": 214494.39 } ] }
+{ "nation_key": 23, "sum_price": [ { "orderdate": "1993-06-08", "sum_price": 161307.05 }, { "orderdate": "1995-12-07", "sum_price": 153048.74 }, { "orderdate": "1994-08-22", "sum_price": 147071.86 } ] }